github.com/cockroachdb/pebble@v1.1.2/replay/replay.go (about) 1 // Copyright 2022 The LevelDB-Go and Pebble Authors. All rights reserved. Use 2 // of this source code is governed by a BSD-style license that can be found in 3 // the LICENSE file. 4 5 // Package replay implements collection and replaying of compaction benchmarking 6 // workloads. A workload is a collection of flushed and ingested sstables, along 7 // with the corresponding manifests describing the order and grouping with which 8 // they were applied. Replaying a workload flushes and ingests the same keys and 9 // sstables to reproduce the write workload for the purpose of evaluating 10 // compaction heuristics. 11 package replay 12 13 import ( 14 "context" 15 "encoding/binary" 16 "fmt" 17 "io" 18 "os" 19 "sort" 20 "strings" 21 "sync" 22 "sync/atomic" 23 "time" 24 25 "github.com/cockroachdb/errors" 26 "github.com/cockroachdb/pebble" 27 "github.com/cockroachdb/pebble/internal/base" 28 "github.com/cockroachdb/pebble/internal/bytealloc" 29 "github.com/cockroachdb/pebble/internal/manifest" 30 "github.com/cockroachdb/pebble/internal/rangedel" 31 "github.com/cockroachdb/pebble/internal/rangekey" 32 "github.com/cockroachdb/pebble/record" 33 "github.com/cockroachdb/pebble/sstable" 34 "github.com/cockroachdb/pebble/vfs" 35 "golang.org/x/perf/benchfmt" 36 "golang.org/x/sync/errgroup" 37 ) 38 39 // A Pacer paces replay of a workload, determining when to apply the next 40 // incoming write. 41 type Pacer interface { 42 pace(r *Runner, step workloadStep) time.Duration 43 } 44 45 // computeReadAmp calculates the read amplification from a manifest.Version 46 func computeReadAmp(v *manifest.Version) int { 47 refRAmp := v.L0Sublevels.ReadAmplification() 48 for _, lvl := range v.Levels[1:] { 49 if !lvl.Empty() { 50 refRAmp++ 51 } 52 } 53 return refRAmp 54 } 55 56 // waitForReadAmpLE is a common function used by PaceByReferenceReadAmp and 57 // PaceByFixedReadAmp to wait on the dbMetricsNotifier condition variable if the 58 // read amplification observed is greater than the specified target (refRAmp). 59 func waitForReadAmpLE(r *Runner, rAmp int) { 60 r.dbMetricsCond.L.Lock() 61 m := r.dbMetrics 62 ra := m.ReadAmp() 63 for ra > rAmp { 64 r.dbMetricsCond.Wait() 65 ra = r.dbMetrics.ReadAmp() 66 } 67 r.dbMetricsCond.L.Unlock() 68 } 69 70 // Unpaced implements Pacer by applying each new write as soon as possible. It 71 // may be useful for examining performance under high read amplification. 72 type Unpaced struct{} 73 74 func (Unpaced) pace(*Runner, workloadStep) (d time.Duration) { return } 75 76 // PaceByReferenceReadAmp implements Pacer by applying each new write following 77 // the collected workloads read amplification. 78 type PaceByReferenceReadAmp struct{} 79 80 func (PaceByReferenceReadAmp) pace(r *Runner, w workloadStep) time.Duration { 81 startTime := time.Now() 82 refRAmp := computeReadAmp(w.pv) 83 waitForReadAmpLE(r, refRAmp) 84 return time.Since(startTime) 85 } 86 87 // PaceByFixedReadAmp implements Pacer by applying each new write following a 88 // fixed read amplification. 89 type PaceByFixedReadAmp int 90 91 func (pra PaceByFixedReadAmp) pace(r *Runner, _ workloadStep) time.Duration { 92 startTime := time.Now() 93 waitForReadAmpLE(r, int(pra)) 94 return time.Since(startTime) 95 } 96 97 // Metrics holds the various statistics on a replay run and its performance. 98 type Metrics struct { 99 CompactionCounts struct { 100 Total int64 101 Default int64 102 DeleteOnly int64 103 ElisionOnly int64 104 Move int64 105 Read int64 106 Rewrite int64 107 MultiLevel int64 108 } 109 EstimatedDebt SampledMetric 110 Final *pebble.Metrics 111 Ingest struct { 112 BytesIntoL0 uint64 113 // BytesWeightedByLevel is calculated as the number of bytes ingested 114 // into a level multiplied by the level's distance from the bottommost 115 // level (L6), summed across all levels. It can be used to guage how 116 // effective heuristics are at ingesting files into lower levels, saving 117 // write amplification. 118 BytesWeightedByLevel uint64 119 } 120 // PaceDuration is the time waiting for the pacer to allow the workload to 121 // continue. 122 PaceDuration time.Duration 123 ReadAmp SampledMetric 124 // QuiesceDuration is the time between completing application of the workload and 125 // compactions quiescing. 126 QuiesceDuration time.Duration 127 TombstoneCount SampledMetric 128 // TotalSize holds the total size of the database, sampled after each 129 // workload step. 130 TotalSize SampledMetric 131 TotalWriteAmp float64 132 WorkloadDuration time.Duration 133 WriteBytes uint64 134 WriteStalls map[string]int 135 WriteStallsDuration map[string]time.Duration 136 WriteThroughput SampledMetric 137 } 138 139 // Plot holds an ascii plot and its name. 140 type Plot struct { 141 Name string 142 Plot string 143 } 144 145 // Plots returns a slice of ascii plots describing metrics change over time. 146 func (m *Metrics) Plots(width, height int) []Plot { 147 const scaleMB = 1.0 / float64(1<<20) 148 return []Plot{ 149 {Name: "Write throughput (MB/s)", Plot: m.WriteThroughput.PlotIncreasingPerSec(width, height, scaleMB)}, 150 {Name: "Estimated compaction debt (MB)", Plot: m.EstimatedDebt.Plot(width, height, scaleMB)}, 151 {Name: "Total database size (MB)", Plot: m.TotalSize.Plot(width, height, scaleMB)}, 152 {Name: "ReadAmp", Plot: m.ReadAmp.Plot(width, height, 1.0)}, 153 } 154 } 155 156 // WriteBenchmarkString writes the metrics in the form of a series of 157 // 'Benchmark' lines understandable by benchstat. 158 func (m *Metrics) WriteBenchmarkString(name string, w io.Writer) error { 159 type benchmarkSection struct { 160 label string 161 values []benchfmt.Value 162 } 163 groups := []benchmarkSection{ 164 {label: "CompactionCounts", values: []benchfmt.Value{ 165 {Value: float64(m.CompactionCounts.Total), Unit: "compactions"}, 166 {Value: float64(m.CompactionCounts.Default), Unit: "default"}, 167 {Value: float64(m.CompactionCounts.DeleteOnly), Unit: "delete"}, 168 {Value: float64(m.CompactionCounts.ElisionOnly), Unit: "elision"}, 169 {Value: float64(m.CompactionCounts.Move), Unit: "move"}, 170 {Value: float64(m.CompactionCounts.Read), Unit: "read"}, 171 {Value: float64(m.CompactionCounts.Rewrite), Unit: "rewrite"}, 172 {Value: float64(m.CompactionCounts.MultiLevel), Unit: "multilevel"}, 173 }}, 174 // Total database sizes sampled after every workload step and 175 // compaction. This can be used to evaluate the relative LSM space 176 // amplification between runs of the same workload. Calculating the true 177 // space amplification continuously is prohibitvely expensive (it 178 // requires totally compacting a copy of the LSM). 179 {label: "DatabaseSize/mean", values: []benchfmt.Value{ 180 {Value: m.TotalSize.Mean(), Unit: "bytes"}, 181 }}, 182 {label: "DatabaseSize/max", values: []benchfmt.Value{ 183 {Value: float64(m.TotalSize.Max()), Unit: "bytes"}, 184 }}, 185 // Time applying the workload and time waiting for compactions to 186 // quiesce after the workload has completed. 187 {label: "DurationWorkload", values: []benchfmt.Value{ 188 {Value: m.WorkloadDuration.Seconds(), Unit: "sec/op"}, 189 }}, 190 {label: "DurationQuiescing", values: []benchfmt.Value{ 191 {Value: m.QuiesceDuration.Seconds(), Unit: "sec/op"}, 192 }}, 193 {label: "DurationPaceDelay", values: []benchfmt.Value{ 194 {Value: m.PaceDuration.Seconds(), Unit: "sec/op"}, 195 }}, 196 // Estimated compaction debt, sampled after every workload step and 197 // compaction. 198 {label: "EstimatedDebt/mean", values: []benchfmt.Value{ 199 {Value: m.EstimatedDebt.Mean(), Unit: "bytes"}, 200 }}, 201 {label: "EstimatedDebt/max", values: []benchfmt.Value{ 202 {Value: float64(m.EstimatedDebt.Max()), Unit: "bytes"}, 203 }}, 204 {label: "FlushUtilization", values: []benchfmt.Value{ 205 {Value: m.Final.Flush.WriteThroughput.Utilization(), Unit: "util"}, 206 }}, 207 {label: "IngestedIntoL0", values: []benchfmt.Value{ 208 {Value: float64(m.Ingest.BytesIntoL0), Unit: "bytes"}, 209 }}, 210 {label: "IngestWeightedByLevel", values: []benchfmt.Value{ 211 {Value: float64(m.Ingest.BytesWeightedByLevel), Unit: "bytes"}, 212 }}, 213 {label: "ReadAmp/mean", values: []benchfmt.Value{ 214 {Value: m.ReadAmp.Mean(), Unit: "files"}, 215 }}, 216 {label: "ReadAmp/max", values: []benchfmt.Value{ 217 {Value: float64(m.ReadAmp.Max()), Unit: "files"}, 218 }}, 219 {label: "TombstoneCount/mean", values: []benchfmt.Value{ 220 {Value: m.TombstoneCount.Mean(), Unit: "tombstones"}, 221 }}, 222 {label: "TombstoneCount/max", values: []benchfmt.Value{ 223 {Value: float64(m.TombstoneCount.Max()), Unit: "tombstones"}, 224 }}, 225 {label: "Throughput", values: []benchfmt.Value{ 226 {Value: float64(m.WriteBytes) / (m.WorkloadDuration + m.QuiesceDuration).Seconds(), Unit: "B/s"}, 227 }}, 228 {label: "WriteAmp", values: []benchfmt.Value{ 229 {Value: float64(m.TotalWriteAmp), Unit: "wamp"}, 230 }}, 231 } 232 233 for _, reason := range []string{"L0", "memtable"} { 234 groups = append(groups, benchmarkSection{ 235 label: fmt.Sprintf("WriteStall/%s", reason), 236 values: []benchfmt.Value{ 237 {Value: float64(m.WriteStalls[reason]), Unit: "stalls"}, 238 {Value: float64(m.WriteStallsDuration[reason].Seconds()), Unit: "stallsec/op"}, 239 }, 240 }) 241 } 242 243 bw := benchfmt.NewWriter(w) 244 for _, grp := range groups { 245 err := bw.Write(&benchfmt.Result{ 246 Name: benchfmt.Name(fmt.Sprintf("BenchmarkReplay/%s/%s", name, grp.label)), 247 Iters: 1, 248 Values: grp.values, 249 }) 250 if err != nil { 251 return err 252 } 253 } 254 return nil 255 } 256 257 // Runner runs a captured workload against a test database, collecting 258 // metrics on performance. 259 type Runner struct { 260 RunDir string 261 WorkloadFS vfs.FS 262 WorkloadPath string 263 Pacer Pacer 264 Opts *pebble.Options 265 MaxWriteBytes uint64 266 267 // Internal state. 268 269 d *pebble.DB 270 // dbMetrics and dbMetricsCond work in unison to update the metrics and 271 // notify (broadcast) to any waiting clients that metrics have been updated. 272 dbMetrics *pebble.Metrics 273 dbMetricsCond sync.Cond 274 cancel func() 275 err atomic.Value 276 errgroup *errgroup.Group 277 readerOpts sstable.ReaderOptions 278 stagingDir string 279 steps chan workloadStep 280 stepsApplied chan workloadStep 281 282 metrics struct { 283 estimatedDebt SampledMetric 284 quiesceDuration time.Duration 285 readAmp SampledMetric 286 tombstoneCount SampledMetric 287 totalSize SampledMetric 288 paceDurationNano atomic.Uint64 289 workloadDuration time.Duration 290 writeBytes atomic.Uint64 291 writeThroughput SampledMetric 292 } 293 writeStallMetrics struct { 294 sync.Mutex 295 countByReason map[string]int 296 durationByReason map[string]time.Duration 297 } 298 // compactionMu holds state for tracking the number of compactions 299 // started and completed and waking waiting goroutines when a new compaction 300 // completes. See nextCompactionCompletes. 301 compactionMu struct { 302 sync.Mutex 303 ch chan struct{} 304 started int64 305 completed int64 306 } 307 workload struct { 308 manifests []string 309 // manifest{Idx,Off} record the starting position of the workload 310 // relative to the initial database state. 311 manifestIdx int 312 manifestOff int64 313 // sstables records the set of captured workload sstables by file num. 314 sstables map[base.FileNum]struct{} 315 } 316 } 317 318 // Run begins executing the workload and returns. 319 // 320 // The workload application will respect the provided context's cancellation. 321 func (r *Runner) Run(ctx context.Context) error { 322 // Find the workload start relative to the RunDir's existing database state. 323 // A prefix of the workload's manifest edits are expected to have already 324 // been applied to the checkpointed existing database state. 325 var err error 326 r.workload.manifests, r.workload.sstables, err = findWorkloadFiles(r.WorkloadPath, r.WorkloadFS) 327 if err != nil { 328 return err 329 } 330 r.workload.manifestIdx, r.workload.manifestOff, err = findManifestStart(r.RunDir, r.Opts.FS, r.workload.manifests) 331 if err != nil { 332 return err 333 } 334 335 // Set up a staging dir for files that will be ingested. 336 r.stagingDir = r.Opts.FS.PathJoin(r.RunDir, "staging") 337 if err := r.Opts.FS.MkdirAll(r.stagingDir, os.ModePerm); err != nil { 338 return err 339 } 340 341 r.dbMetricsCond = sync.Cond{ 342 L: &sync.Mutex{}, 343 } 344 345 // Extend the user-provided Options with extensions necessary for replay 346 // mechanics. 347 r.compactionMu.ch = make(chan struct{}) 348 r.Opts.AddEventListener(r.eventListener()) 349 r.writeStallMetrics.countByReason = make(map[string]int) 350 r.writeStallMetrics.durationByReason = make(map[string]time.Duration) 351 r.Opts.EnsureDefaults() 352 r.readerOpts = r.Opts.MakeReaderOptions() 353 r.Opts.DisableWAL = true 354 r.d, err = pebble.Open(r.RunDir, r.Opts) 355 if err != nil { 356 return err 357 } 358 359 r.dbMetrics = r.d.Metrics() 360 361 // Use a buffered channel to allow the prepareWorkloadSteps to read ahead, 362 // buffering up to cap(r.steps) steps ahead of the current applied state. 363 // Flushes need to be buffered and ingested sstables need to be copied, so 364 // pipelining this preparation makes it more likely the step will be ready 365 // to apply when the pacer decides to apply it. 366 r.steps = make(chan workloadStep, 5) 367 r.stepsApplied = make(chan workloadStep, 5) 368 369 ctx, r.cancel = context.WithCancel(ctx) 370 r.errgroup, ctx = errgroup.WithContext(ctx) 371 r.errgroup.Go(func() error { return r.prepareWorkloadSteps(ctx) }) 372 r.errgroup.Go(func() error { return r.applyWorkloadSteps(ctx) }) 373 r.errgroup.Go(func() error { return r.refreshMetrics(ctx) }) 374 return nil 375 } 376 377 // refreshMetrics runs in its own goroutine, collecting metrics from the Pebble 378 // instance whenever a) a workload step completes, or b) a compaction completes. 379 // The Pacer implementations that pace based on read-amplification rely on these 380 // refreshed metrics to decide when to allow the workload to proceed. 381 func (r *Runner) refreshMetrics(ctx context.Context) error { 382 startAt := time.Now() 383 var workloadExhausted bool 384 var workloadExhaustedAt time.Time 385 stepsApplied := r.stepsApplied 386 compactionCount, alreadyCompleted, compactionCh := r.nextCompactionCompletes(0) 387 for { 388 if !alreadyCompleted { 389 select { 390 case <-ctx.Done(): 391 return ctx.Err() 392 case <-compactionCh: 393 // Fall through to refreshing dbMetrics. 394 case _, ok := <-stepsApplied: 395 if !ok { 396 workloadExhausted = true 397 workloadExhaustedAt = time.Now() 398 // Set the [stepsApplied] channel to nil so that we'll never 399 // hit this case again, and we don't busy loop. 400 stepsApplied = nil 401 // Record the replay time. 402 r.metrics.workloadDuration = workloadExhaustedAt.Sub(startAt) 403 } 404 // Fall through to refreshing dbMetrics. 405 } 406 } 407 408 m := r.d.Metrics() 409 r.dbMetricsCond.L.Lock() 410 r.dbMetrics = m 411 r.dbMetricsCond.Broadcast() 412 r.dbMetricsCond.L.Unlock() 413 414 // Collect sample metrics. These metrics are calculated by sampling 415 // every time we collect metrics. 416 r.metrics.readAmp.record(int64(m.ReadAmp())) 417 r.metrics.estimatedDebt.record(int64(m.Compact.EstimatedDebt)) 418 r.metrics.tombstoneCount.record(int64(m.Keys.TombstoneCount)) 419 r.metrics.totalSize.record(int64(m.DiskSpaceUsage())) 420 r.metrics.writeThroughput.record(int64(r.metrics.writeBytes.Load())) 421 422 compactionCount, alreadyCompleted, compactionCh = r.nextCompactionCompletes(compactionCount) 423 // Consider whether replaying is complete. There are two necessary 424 // conditions: 425 // 426 // 1. The workload must be exhausted. 427 // 2. Compactions must have quiesced. 428 // 429 // The first condition is simple. The replay tool is responsible for 430 // applying the workload. The goroutine responsible for applying the 431 // workload closes the `stepsApplied` channel after the last step has 432 // been applied, and we'll flip `workloadExhausted` to true. 433 // 434 // The second condition is tricky. The replay tool doesn't control 435 // compactions and doesn't have visibility into whether the compaction 436 // picker is about to schedule a new compaction. We can tell when 437 // compactions are in progress or may be immeninent (eg, flushes in 438 // progress). If it appears that compactions have quiesced, pause for a 439 // fixed duration to see if a new one is scheduled. If not, consider 440 // compactions quiesced. 441 if workloadExhausted && !alreadyCompleted && r.compactionsAppearQuiesced(m) { 442 select { 443 case <-compactionCh: 444 // A new compaction just finished; compactions have not 445 // quiesced. 446 continue 447 case <-time.After(time.Second): 448 // No compactions completed. If it still looks like they've 449 // quiesced according to the metrics, consider them quiesced. 450 if r.compactionsAppearQuiesced(r.d.Metrics()) { 451 r.metrics.quiesceDuration = time.Since(workloadExhaustedAt) 452 return nil 453 } 454 } 455 } 456 } 457 } 458 459 // compactionsAppearQuiesced returns true if the database may have quiesced, and 460 // there likely won't be additional compactions scheduled. Detecting quiescence 461 // is a bit fraught: The various signals that Pebble makes available are 462 // adjusted at different points in the compaction lifecycle, and database 463 // mutexes are dropped and acquired between them. This makes it difficult to 464 // reliably identify when compactions quiesce. 465 // 466 // For example, our call to DB.Metrics() may acquire the DB.mu mutex when a 467 // compaction has just successfully completed, but before it's managed to 468 // schedule the next compaction (DB.mu is dropped while it attempts to acquire 469 // the manifest lock). 470 func (r *Runner) compactionsAppearQuiesced(m *pebble.Metrics) bool { 471 r.compactionMu.Lock() 472 defer r.compactionMu.Unlock() 473 if m.Flush.NumInProgress > 0 { 474 return false 475 } else if m.Compact.NumInProgress > 0 && r.compactionMu.started != r.compactionMu.completed { 476 return false 477 } 478 return true 479 } 480 481 // nextCompactionCompletes may be used to be notified when new compactions 482 // complete. The caller is responsible for holding on to a monotonically 483 // increasing count representing the number of compactions that have been 484 // observed, beginning at zero. 485 // 486 // The caller passes their current count as an argument. If a new compaction has 487 // already completed since their provided count, nextCompactionCompletes returns 488 // the new count and a true boolean return value. If a new compaction has not 489 // yet completed, it returns a channel that will be closed when the next 490 // compaction completes. This scheme allows the caller to select{...}, 491 // performing some action on every compaction completion. 492 func (r *Runner) nextCompactionCompletes( 493 lastObserved int64, 494 ) (count int64, alreadyOccurred bool, ch chan struct{}) { 495 r.compactionMu.Lock() 496 defer r.compactionMu.Unlock() 497 498 if lastObserved < r.compactionMu.completed { 499 // There has already been another compaction since the last one observed 500 // by this caller. Return immediately. 501 return r.compactionMu.completed, true, nil 502 } 503 504 // The last observed compaction is still the most recent compaction. 505 // Return a channel that the caller can wait on to be notified when the 506 // next compaction occurs. 507 if r.compactionMu.ch == nil { 508 r.compactionMu.ch = make(chan struct{}) 509 } 510 return lastObserved, false, r.compactionMu.ch 511 } 512 513 // Wait waits for the workload replay to complete. Wait returns once the entire 514 // workload has been replayed, and compactions have quiesced. 515 func (r *Runner) Wait() (Metrics, error) { 516 err := r.errgroup.Wait() 517 if storedErr := r.err.Load(); storedErr != nil { 518 err = storedErr.(error) 519 } 520 pm := r.d.Metrics() 521 total := pm.Total() 522 var ingestBytesWeighted uint64 523 for l := 0; l < len(pm.Levels); l++ { 524 ingestBytesWeighted += pm.Levels[l].BytesIngested * uint64(len(pm.Levels)-l-1) 525 } 526 527 m := Metrics{ 528 Final: pm, 529 EstimatedDebt: r.metrics.estimatedDebt, 530 PaceDuration: time.Duration(r.metrics.paceDurationNano.Load()), 531 ReadAmp: r.metrics.readAmp, 532 QuiesceDuration: r.metrics.quiesceDuration, 533 TombstoneCount: r.metrics.tombstoneCount, 534 TotalSize: r.metrics.totalSize, 535 TotalWriteAmp: total.WriteAmp(), 536 WorkloadDuration: r.metrics.workloadDuration, 537 WriteBytes: r.metrics.writeBytes.Load(), 538 WriteStalls: make(map[string]int), 539 WriteStallsDuration: make(map[string]time.Duration), 540 WriteThroughput: r.metrics.writeThroughput, 541 } 542 543 r.writeStallMetrics.Lock() 544 for reason, count := range r.writeStallMetrics.countByReason { 545 m.WriteStalls[reason] = count 546 } 547 for reason, duration := range r.writeStallMetrics.durationByReason { 548 m.WriteStallsDuration[reason] = duration 549 } 550 r.writeStallMetrics.Unlock() 551 m.CompactionCounts.Total = pm.Compact.Count 552 m.CompactionCounts.Default = pm.Compact.DefaultCount 553 m.CompactionCounts.DeleteOnly = pm.Compact.DeleteOnlyCount 554 m.CompactionCounts.ElisionOnly = pm.Compact.ElisionOnlyCount 555 m.CompactionCounts.Move = pm.Compact.MoveCount 556 m.CompactionCounts.Read = pm.Compact.ReadCount 557 m.CompactionCounts.Rewrite = pm.Compact.RewriteCount 558 m.CompactionCounts.MultiLevel = pm.Compact.MultiLevelCount 559 m.Ingest.BytesIntoL0 = pm.Levels[0].BytesIngested 560 m.Ingest.BytesWeightedByLevel = ingestBytesWeighted 561 return m, err 562 } 563 564 // Close closes remaining open resources, including the database. It must be 565 // called after Wait. 566 func (r *Runner) Close() error { 567 return r.d.Close() 568 } 569 570 // A workloadStep describes a single manifest edit in the workload. It may be a 571 // flush or ingest that should be applied to the test database, or it may be a 572 // compaction that is surfaced to allow the replay logic to compare against the 573 // state of the database at workload collection time. 574 type workloadStep struct { 575 kind stepKind 576 ve manifest.VersionEdit 577 // a Version describing the state of the LSM *before* the workload was 578 // collected. 579 pv *manifest.Version 580 // a Version describing the state of the LSM when the workload was 581 // collected. 582 v *manifest.Version 583 // non-nil for flushStepKind 584 flushBatch *pebble.Batch 585 tablesToIngest []string 586 cumulativeWriteBytes uint64 587 } 588 589 type stepKind uint8 590 591 const ( 592 flushStepKind stepKind = iota 593 ingestStepKind 594 compactionStepKind 595 ) 596 597 // eventListener returns a Pebble EventListener that is installed on the replay 598 // database so that the replay runner has access to internal Pebble events. 599 func (r *Runner) eventListener() pebble.EventListener { 600 var writeStallBegin time.Time 601 var writeStallReason string 602 l := pebble.EventListener{ 603 BackgroundError: func(err error) { 604 r.err.Store(err) 605 r.cancel() 606 }, 607 WriteStallBegin: func(info pebble.WriteStallBeginInfo) { 608 r.writeStallMetrics.Lock() 609 defer r.writeStallMetrics.Unlock() 610 writeStallReason = info.Reason 611 // Take just the first word of the reason. 612 if j := strings.IndexByte(writeStallReason, ' '); j != -1 { 613 writeStallReason = writeStallReason[:j] 614 } 615 switch writeStallReason { 616 case "L0", "memtable": 617 r.writeStallMetrics.countByReason[writeStallReason]++ 618 default: 619 panic(fmt.Sprintf("unrecognized write stall reason %q", info.Reason)) 620 } 621 writeStallBegin = time.Now() 622 }, 623 WriteStallEnd: func() { 624 r.writeStallMetrics.Lock() 625 defer r.writeStallMetrics.Unlock() 626 r.writeStallMetrics.durationByReason[writeStallReason] += time.Since(writeStallBegin) 627 }, 628 CompactionBegin: func(_ pebble.CompactionInfo) { 629 r.compactionMu.Lock() 630 defer r.compactionMu.Unlock() 631 r.compactionMu.started++ 632 }, 633 CompactionEnd: func(_ pebble.CompactionInfo) { 634 // Keep track of the number of compactions that complete and notify 635 // anyone waiting for a compaction to complete. See the function 636 // nextCompactionCompletes for the corresponding receiver side. 637 r.compactionMu.Lock() 638 defer r.compactionMu.Unlock() 639 r.compactionMu.completed++ 640 if r.compactionMu.ch != nil { 641 // Signal that a compaction has completed. 642 close(r.compactionMu.ch) 643 r.compactionMu.ch = nil 644 } 645 }, 646 } 647 l.EnsureDefaults(nil) 648 return l 649 } 650 651 // applyWorkloadSteps runs in its own goroutine, reading workload steps off the 652 // r.steps channel and applying them to the test database. 653 func (r *Runner) applyWorkloadSteps(ctx context.Context) error { 654 for { 655 var ok bool 656 var step workloadStep 657 select { 658 case <-ctx.Done(): 659 return ctx.Err() 660 case step, ok = <-r.steps: 661 if !ok { 662 // Exhausted the workload. Exit. 663 close(r.stepsApplied) 664 return nil 665 } 666 } 667 668 paceDur := r.Pacer.pace(r, step) 669 r.metrics.paceDurationNano.Add(uint64(paceDur)) 670 671 switch step.kind { 672 case flushStepKind: 673 if err := step.flushBatch.Commit(&pebble.WriteOptions{Sync: false}); err != nil { 674 return err 675 } 676 _, err := r.d.AsyncFlush() 677 if err != nil { 678 return err 679 } 680 r.metrics.writeBytes.Store(step.cumulativeWriteBytes) 681 r.stepsApplied <- step 682 case ingestStepKind: 683 if err := r.d.Ingest(step.tablesToIngest); err != nil { 684 return err 685 } 686 r.metrics.writeBytes.Store(step.cumulativeWriteBytes) 687 r.stepsApplied <- step 688 case compactionStepKind: 689 // No-op. 690 // TODO(jackson): Should we elide this earlier? 691 default: 692 panic("unreachable") 693 } 694 } 695 } 696 697 // prepareWorkloadSteps runs in its own goroutine, reading the workload 698 // manifests in order to reconstruct the workload and prepare each step to be 699 // applied. It sends each workload step to the r.steps channel. 700 func (r *Runner) prepareWorkloadSteps(ctx context.Context) error { 701 defer func() { close(r.steps) }() 702 703 idx := r.workload.manifestIdx 704 705 var cumulativeWriteBytes uint64 706 var flushBufs flushBuffers 707 var v *manifest.Version 708 var previousVersion *manifest.Version 709 var bve manifest.BulkVersionEdit 710 bve.AddedByFileNum = make(map[base.FileNum]*manifest.FileMetadata) 711 applyVE := func(ve *manifest.VersionEdit) error { 712 return bve.Accumulate(ve) 713 } 714 currentVersion := func() (*manifest.Version, error) { 715 var err error 716 v, err = bve.Apply(v, 717 r.Opts.Comparer.Compare, 718 r.Opts.Comparer.FormatKey, 719 r.Opts.FlushSplitBytes, 720 r.Opts.Experimental.ReadCompactionRate, 721 nil, /* zombies */ 722 manifest.ProhibitSplitUserKeys) 723 bve = manifest.BulkVersionEdit{AddedByFileNum: bve.AddedByFileNum} 724 return v, err 725 } 726 727 for ; idx < len(r.workload.manifests); idx++ { 728 if r.MaxWriteBytes != 0 && cumulativeWriteBytes > r.MaxWriteBytes { 729 break 730 } 731 732 err := func() error { 733 manifestName := r.workload.manifests[idx] 734 f, err := r.WorkloadFS.Open(r.WorkloadFS.PathJoin(r.WorkloadPath, manifestName)) 735 if err != nil { 736 return err 737 } 738 defer f.Close() 739 740 rr := record.NewReader(f, 0 /* logNum */) 741 // A manifest's first record always holds the initial version state. 742 // If this is the first manifest we're examining, we load it in 743 // order to seed `metas` with the file metadata of the existing 744 // files. Otherwise, we can skip it because we already know all the 745 // file metadatas up to this point. 746 rec, err := rr.Next() 747 if err != nil { 748 return err 749 } 750 if idx == r.workload.manifestIdx { 751 var ve manifest.VersionEdit 752 if err := ve.Decode(rec); err != nil { 753 return err 754 } 755 if err := applyVE(&ve); err != nil { 756 return err 757 } 758 } 759 760 // Read the remaining of the manifests version edits, one-by-one. 761 for { 762 rec, err := rr.Next() 763 if err == io.EOF || record.IsInvalidRecord(err) { 764 break 765 } else if err != nil { 766 return err 767 } 768 var ve manifest.VersionEdit 769 if err = ve.Decode(rec); err == io.EOF || record.IsInvalidRecord(err) { 770 break 771 } else if err != nil { 772 return err 773 } 774 if err := applyVE(&ve); err != nil { 775 return err 776 } 777 if idx == r.workload.manifestIdx && rr.Offset() <= r.workload.manifestOff { 778 // The record rec began at an offset strictly less than 779 // rr.Offset(), which means it's strictly less than 780 // r.workload.manifestOff, and we should skip it. 781 continue 782 } 783 if len(ve.NewFiles) == 0 && len(ve.DeletedFiles) == 0 { 784 // Skip WAL rotations and other events that don't affect the 785 // files of the LSM. 786 continue 787 } 788 789 s := workloadStep{ve: ve} 790 if len(ve.DeletedFiles) > 0 { 791 // If a version edit deletes files, we assume it's a compaction. 792 s.kind = compactionStepKind 793 } else { 794 // Default to ingest. If any files have unequal 795 // smallest,largest sequence numbers, we'll update this to a 796 // flush. 797 s.kind = ingestStepKind 798 } 799 var newFiles []base.DiskFileNum 800 for _, nf := range ve.NewFiles { 801 newFiles = append(newFiles, nf.Meta.FileBacking.DiskFileNum) 802 if s.kind == ingestStepKind && (nf.Meta.SmallestSeqNum != nf.Meta.LargestSeqNum || nf.Level != 0) { 803 s.kind = flushStepKind 804 } 805 } 806 // Add the current reference *Version to the step. This provides 807 // access to, for example, the read-amplification of the 808 // database at this point when the workload was collected. This 809 // can be useful for pacing. 810 if s.v, err = currentVersion(); err != nil { 811 return err 812 } 813 // On the first time through, we set the previous version to the current 814 // version otherwise we set it to the actual previous version. 815 if previousVersion == nil { 816 previousVersion = s.v 817 } 818 s.pv = previousVersion 819 previousVersion = s.v 820 821 // It's possible that the workload collector captured this 822 // version edit, but wasn't able to collect all of the 823 // corresponding sstables before being terminated. 824 if s.kind == flushStepKind || s.kind == ingestStepKind { 825 for _, fileNum := range newFiles { 826 if _, ok := r.workload.sstables[fileNum.FileNum()]; !ok { 827 // TODO(jackson,leon): This isn't exactly an error 828 // condition. Give this more thought; do we want to 829 // require graceful exiting of workload collection, 830 // such that the last version edit must have had its 831 // corresponding sstables collected? 832 return errors.Newf("sstable %s not found", fileNum) 833 } 834 } 835 } 836 837 switch s.kind { 838 case flushStepKind: 839 // Load all of the flushed sstables' keys into a batch. 840 s.flushBatch = r.d.NewBatch() 841 if err := loadFlushedSSTableKeys(s.flushBatch, r.WorkloadFS, r.WorkloadPath, newFiles, r.readerOpts, &flushBufs); err != nil { 842 return errors.Wrapf(err, "flush in %q at offset %d", manifestName, rr.Offset()) 843 } 844 cumulativeWriteBytes += uint64(s.flushBatch.Len()) 845 case ingestStepKind: 846 // Copy the ingested sstables into a staging area within the 847 // run dir. This is necessary for two reasons: 848 // a) Ingest will remove the source file, and we don't want 849 // to mutate the workload. 850 // b) If the workload stored on another volume, Ingest 851 // would need to fall back to copying the file since 852 // it's not possible to link across volumes. The true 853 // workload likely linked the file. Staging the file 854 // ahead of time ensures that we're able to Link the 855 // file like the original workload did. 856 for _, fileNum := range newFiles { 857 src := base.MakeFilepath(r.WorkloadFS, r.WorkloadPath, base.FileTypeTable, fileNum) 858 dst := base.MakeFilepath(r.Opts.FS, r.stagingDir, base.FileTypeTable, fileNum) 859 if err := vfs.CopyAcrossFS(r.WorkloadFS, src, r.Opts.FS, dst); err != nil { 860 return errors.Wrapf(err, "ingest in %q at offset %d", manifestName, rr.Offset()) 861 } 862 finfo, err := r.Opts.FS.Stat(dst) 863 if err != nil { 864 return errors.Wrapf(err, "stating %q", dst) 865 } 866 cumulativeWriteBytes += uint64(finfo.Size()) 867 s.tablesToIngest = append(s.tablesToIngest, dst) 868 } 869 case compactionStepKind: 870 // Nothing to do. 871 } 872 s.cumulativeWriteBytes = cumulativeWriteBytes 873 874 select { 875 case <-ctx.Done(): 876 return ctx.Err() 877 case r.steps <- s: 878 } 879 880 if r.MaxWriteBytes != 0 && cumulativeWriteBytes > r.MaxWriteBytes { 881 break 882 } 883 } 884 return nil 885 }() 886 if err != nil { 887 return err 888 } 889 } 890 return nil 891 } 892 893 // findWorkloadFiles finds all manifests and tables in the provided path on fs. 894 func findWorkloadFiles( 895 path string, fs vfs.FS, 896 ) (manifests []string, sstables map[base.FileNum]struct{}, err error) { 897 dirents, err := fs.List(path) 898 if err != nil { 899 return nil, nil, err 900 } 901 sstables = make(map[base.FileNum]struct{}) 902 for _, dirent := range dirents { 903 typ, fileNum, ok := base.ParseFilename(fs, dirent) 904 if !ok { 905 continue 906 } 907 switch typ { 908 case base.FileTypeManifest: 909 manifests = append(manifests, dirent) 910 case base.FileTypeTable: 911 sstables[fileNum.FileNum()] = struct{}{} 912 } 913 } 914 if len(manifests) == 0 { 915 return nil, nil, errors.Newf("no manifests found") 916 } 917 sort.Strings(manifests) 918 return manifests, sstables, err 919 } 920 921 // findManifestStart takes a database directory and FS containing the initial 922 // database state that a workload will be run against, and a list of a workloads 923 // manifests. It examines the database's current manifest to determine where 924 // workload replay should begin, so as to not duplicate already-applied version 925 // edits. 926 // 927 // It returns the index of the starting manifest, and the database's current 928 // offset within the manifest. 929 func findManifestStart( 930 dbDir string, dbFS vfs.FS, manifests []string, 931 ) (index int, offset int64, err error) { 932 // Identify the database's current manifest. 933 dbDesc, err := pebble.Peek(dbDir, dbFS) 934 if err != nil { 935 return 0, 0, err 936 } 937 dbManifest := dbFS.PathBase(dbDesc.ManifestFilename) 938 // If there is no initial database state, begin workload replay from the 939 // beginning of the first manifest. 940 if !dbDesc.Exists { 941 return 0, 0, nil 942 } 943 for index = 0; index < len(manifests); index++ { 944 if manifests[index] == dbManifest { 945 break 946 } 947 } 948 if index == len(manifests) { 949 // The initial database state has a manifest that does not appear within 950 // the workload's set of manifests. This is possible if we began 951 // recording the workload at the same time as a manifest rotation, but 952 // more likely we're applying a workload to a different initial database 953 // state than the one from which the workload was collected. Either way, 954 // start from the beginning of the first manifest. 955 return 0, 0, nil 956 } 957 // Find the initial database's offset within the manifest. 958 info, err := dbFS.Stat(dbFS.PathJoin(dbDir, dbManifest)) 959 if err != nil { 960 return 0, 0, err 961 } 962 return index, info.Size(), nil 963 } 964 965 // loadFlushedSSTableKeys copies keys from the sstables specified by `fileNums` 966 // in the directory specified by `path` into the provided the batch. Keys are 967 // applied to the batch in the order dictated by their sequence numbers within 968 // the sstables, ensuring the relative relationship between sequence numbers is 969 // maintained. 970 // 971 // Preserving the relative relationship between sequence numbers is not strictly 972 // necessary, but it ensures we accurately exercise some microoptimizations (eg, 973 // detecting user key changes by descending trailer). There may be additional 974 // dependencies on sequence numbers in the future. 975 func loadFlushedSSTableKeys( 976 b *pebble.Batch, 977 fs vfs.FS, 978 path string, 979 fileNums []base.DiskFileNum, 980 readOpts sstable.ReaderOptions, 981 bufs *flushBuffers, 982 ) error { 983 // Load all the keys across all the sstables. 984 for _, fileNum := range fileNums { 985 if err := func() error { 986 filePath := base.MakeFilepath(fs, path, base.FileTypeTable, fileNum) 987 f, err := fs.Open(filePath) 988 if err != nil { 989 return err 990 } 991 readable, err := sstable.NewSimpleReadable(f) 992 if err != nil { 993 f.Close() 994 return err 995 } 996 r, err := sstable.NewReader(readable, readOpts) 997 if err != nil { 998 return err 999 } 1000 defer r.Close() 1001 1002 // Load all the point keys. 1003 iter, err := r.NewIter(nil, nil) 1004 if err != nil { 1005 return err 1006 } 1007 defer iter.Close() 1008 for k, lv := iter.First(); k != nil; k, lv = iter.Next() { 1009 var key flushedKey 1010 key.Trailer = k.Trailer 1011 bufs.alloc, key.UserKey = bufs.alloc.Copy(k.UserKey) 1012 if v, callerOwned, err := lv.Value(nil); err != nil { 1013 return err 1014 } else if callerOwned { 1015 key.value = v 1016 } else { 1017 bufs.alloc, key.value = bufs.alloc.Copy(v) 1018 } 1019 bufs.keys = append(bufs.keys, key) 1020 } 1021 1022 // Load all the range tombstones. 1023 if iter, err := r.NewRawRangeDelIter(); err != nil { 1024 return err 1025 } else if iter != nil { 1026 defer iter.Close() 1027 for s := iter.First(); s != nil; s = iter.Next() { 1028 if err := rangedel.Encode(s, func(k base.InternalKey, v []byte) error { 1029 var key flushedKey 1030 key.Trailer = k.Trailer 1031 bufs.alloc, key.UserKey = bufs.alloc.Copy(k.UserKey) 1032 bufs.alloc, key.value = bufs.alloc.Copy(v) 1033 bufs.keys = append(bufs.keys, key) 1034 return nil 1035 }); err != nil { 1036 return err 1037 } 1038 } 1039 } 1040 1041 // Load all the range keys. 1042 if iter, err := r.NewRawRangeKeyIter(); err != nil { 1043 return err 1044 } else if iter != nil { 1045 defer iter.Close() 1046 for s := iter.First(); s != nil; s = iter.Next() { 1047 if err := rangekey.Encode(s, func(k base.InternalKey, v []byte) error { 1048 var key flushedKey 1049 key.Trailer = k.Trailer 1050 bufs.alloc, key.UserKey = bufs.alloc.Copy(k.UserKey) 1051 bufs.alloc, key.value = bufs.alloc.Copy(v) 1052 bufs.keys = append(bufs.keys, key) 1053 return nil 1054 }); err != nil { 1055 return err 1056 } 1057 } 1058 } 1059 return nil 1060 }(); err != nil { 1061 return err 1062 } 1063 } 1064 1065 // Sort the flushed keys by their sequence numbers so that we can apply them 1066 // to the batch in the same order, maintaining the relative relationship 1067 // between keys. 1068 // NB: We use a stable sort so that keys corresponding to span fragments 1069 // (eg, range tombstones and range keys) have a deterministic ordering for 1070 // testing. 1071 sort.Stable(bufs.keys) 1072 1073 // Add the keys to the batch in the order they were committed when the 1074 // workload was captured. 1075 for i := 0; i < len(bufs.keys); i++ { 1076 var err error 1077 switch bufs.keys[i].Kind() { 1078 case base.InternalKeyKindDelete: 1079 err = b.Delete(bufs.keys[i].UserKey, nil) 1080 case base.InternalKeyKindDeleteSized: 1081 v, _ := binary.Uvarint(bufs.keys[i].value) 1082 // Batch.DeleteSized takes just the length of the value being 1083 // deleted and adds the key's length to derive the overall entry 1084 // size of the value being deleted. This has already been done to 1085 // the key we're reading from the sstable, so we must subtract the 1086 // key length from the encoded value before calling b.DeleteSized, 1087 // which will again add the key length before encoding. 1088 err = b.DeleteSized(bufs.keys[i].UserKey, uint32(v-uint64(len(bufs.keys[i].UserKey))), nil) 1089 case base.InternalKeyKindSet, base.InternalKeyKindSetWithDelete: 1090 err = b.Set(bufs.keys[i].UserKey, bufs.keys[i].value, nil) 1091 case base.InternalKeyKindMerge: 1092 err = b.Merge(bufs.keys[i].UserKey, bufs.keys[i].value, nil) 1093 case base.InternalKeyKindSingleDelete: 1094 err = b.SingleDelete(bufs.keys[i].UserKey, nil) 1095 case base.InternalKeyKindRangeDelete: 1096 err = b.DeleteRange(bufs.keys[i].UserKey, bufs.keys[i].value, nil) 1097 case base.InternalKeyKindRangeKeySet, base.InternalKeyKindRangeKeyUnset, base.InternalKeyKindRangeKeyDelete: 1098 s, err := rangekey.Decode(bufs.keys[i].InternalKey, bufs.keys[i].value, nil) 1099 if err != nil { 1100 return err 1101 } 1102 if len(s.Keys) != 1 { 1103 return errors.Newf("range key span unexpectedly contains %d keys", len(s.Keys)) 1104 } 1105 switch bufs.keys[i].Kind() { 1106 case base.InternalKeyKindRangeKeySet: 1107 err = b.RangeKeySet(s.Start, s.End, s.Keys[0].Suffix, s.Keys[0].Value, nil) 1108 case base.InternalKeyKindRangeKeyUnset: 1109 err = b.RangeKeyUnset(s.Start, s.End, s.Keys[0].Suffix, nil) 1110 case base.InternalKeyKindRangeKeyDelete: 1111 err = b.RangeKeyDelete(s.Start, s.End, nil) 1112 default: 1113 err = errors.Newf("unexpected key kind %q", bufs.keys[i].Kind()) 1114 } 1115 if err != nil { 1116 return err 1117 } 1118 default: 1119 err = errors.Newf("unexpected key kind %q", bufs.keys[i].Kind()) 1120 } 1121 if err != nil { 1122 return err 1123 } 1124 } 1125 1126 // Done with the flushBuffers. Reset. 1127 bufs.keys = bufs.keys[:0] 1128 return nil 1129 } 1130 1131 type flushBuffers struct { 1132 keys flushedKeysByTrailer 1133 alloc bytealloc.A 1134 } 1135 1136 type flushedKeysByTrailer []flushedKey 1137 1138 func (s flushedKeysByTrailer) Len() int { return len(s) } 1139 func (s flushedKeysByTrailer) Less(i, j int) bool { return s[i].Trailer < s[j].Trailer } 1140 func (s flushedKeysByTrailer) Swap(i, j int) { s[i], s[j] = s[j], s[i] } 1141 1142 type flushedKey struct { 1143 base.InternalKey 1144 value []byte 1145 }