github.com/cockroachdb/pebble@v0.0.0-20231214172447-ab4952c5f87b/tool/logs/compaction.go (about) 1 // Copyright 2021 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 logs 6 7 import ( 8 "bufio" 9 "bytes" 10 "cmp" 11 "fmt" 12 "math" 13 "os" 14 "path/filepath" 15 "regexp" 16 "slices" 17 "sort" 18 "strconv" 19 "strings" 20 "time" 21 22 "github.com/cockroachdb/errors" 23 "github.com/cockroachdb/pebble/internal/humanize" 24 "github.com/cockroachdb/pebble/internal/manifest" 25 "github.com/spf13/cobra" 26 ) 27 28 const numLevels = manifest.NumLevels 29 30 var ( 31 // Captures a common logging prefix that can be used as the context for the 32 // surrounding information captured by other expressions. Example: 33 // 34 // I211215 14:26:56.012382 51831533 3@vendor/github.com/cockroachdb/pebble/compaction.go:1845 ⋮ [T1,n5,pebble,s5] ... 35 // 36 logContextPattern = regexp.MustCompile( 37 `^.*` + 38 /* Timestamp */ `(?P<timestamp>\d{6} \d{2}:\d{2}:\d{2}.\d{6}).*` + 39 /* Node / Store */ `\[(T(\d+|\?),)?n(?P<node>\d+|\?).*,s(?P<store>\d+|\?).*?\].*`, 40 ) 41 logContextPatternTimestampIdx = logContextPattern.SubexpIndex("timestamp") 42 logContextPatternNodeIdx = logContextPattern.SubexpIndex("node") 43 logContextPatternStoreIdx = logContextPattern.SubexpIndex("store") 44 45 // Matches either a compaction or a memtable flush log line. 46 // 47 // A compaction start / end line resembles: 48 // "[JOB X] compact(ed|ing)" 49 // 50 // A memtable flush start / end line resembles: 51 // "[JOB X] flush(ed|ing)" 52 // 53 // An ingested sstable flush looks like: 54 // "[JOB 226] flushed 6 ingested flushables" 55 sentinelPattern = regexp.MustCompile(`\[JOB.*(?P<prefix>compact|flush|ingest)(?P<suffix>ed|ing)[^:]`) 56 sentinelPatternPrefixIdx = sentinelPattern.SubexpIndex("prefix") 57 sentinelPatternSuffixIdx = sentinelPattern.SubexpIndex("suffix") 58 59 // Example compaction start and end log lines: 60 // 23.1 and older: 61 // I211215 14:26:56.012382 51831533 3@vendor/github.com/cockroachdb/pebble/compaction.go:1845 ⋮ [n5,pebble,s5] 1216510 [JOB 284925] compacting(default) L2 [442555] (4.2 M) + L3 [445853] (8.4 M) 62 // I211215 14:26:56.318543 51831533 3@vendor/github.com/cockroachdb/pebble/compaction.go:1886 ⋮ [n5,pebble,s5] 1216554 [JOB 284925] compacted(default) L2 [442555] (4.2 M) + L3 [445853] (8.4 M) -> L3 [445883 445887] (13 M), in 0.3s, output rate 42 M/s 63 // current: 64 // I211215 14:26:56.012382 51831533 3@vendor/github.com/cockroachdb/pebble/compaction.go:1845 ⋮ [n5,pebble,s5] 1216510 [JOB 284925] compacting(default) L2 [442555] (4.2MB) + L3 [445853] (8.4MB) 65 // I211215 14:26:56.318543 51831533 3@vendor/github.com/cockroachdb/pebble/compaction.go:1886 ⋮ [n5,pebble,s5] 1216554 [JOB 284925] compacted(default) L2 [442555] (4.2MB) + L3 [445853] (8.4MB) -> L3 [445883 445887] (13MB), in 0.3s, output rate 42MB/s 66 // 67 // NOTE: we use the log timestamp to compute the compaction duration rather 68 // than the Pebble log output. 69 compactionPattern = regexp.MustCompile( 70 `^.*` + 71 /* Job ID */ `\[JOB (?P<job>\d+)]\s` + 72 /* Start / end */ `compact(?P<suffix>ed|ing)` + 73 74 /* Compaction type */ 75 `\((?P<type>.*?)\)\s` + 76 /* Optional annotation*/ `?(\s*\[(?P<annotations>.*?)\]\s*)?` + 77 78 /* Start / end level */ 79 `(?P<levels>L(?P<from>\d).*?(?:.*(?:\+|->)\sL(?P<to>\d))?` + 80 /* Bytes */ 81 `(?:.*?\((?P<bytes>[0-9.]+( [BKMGTPE]|[KMGTPE]?B))\))` + 82 /* Score */ 83 `?(\s*(Score=\d+(\.\d+)))?)`, 84 ) 85 compactionPatternJobIdx = compactionPattern.SubexpIndex("job") 86 compactionPatternSuffixIdx = compactionPattern.SubexpIndex("suffix") 87 compactionPatternTypeIdx = compactionPattern.SubexpIndex("type") 88 compactionPatternLevels = compactionPattern.SubexpIndex("levels") 89 compactionPatternFromIdx = compactionPattern.SubexpIndex("from") 90 compactionPatternToIdx = compactionPattern.SubexpIndex("to") 91 compactionPatternBytesIdx = compactionPattern.SubexpIndex("bytes") 92 93 // Example memtable flush log lines: 94 // 23.1 and older: 95 // I211213 16:23:48.903751 21136 3@vendor/github.com/cockroachdb/pebble/event.go:599 ⋮ [n9,pebble,s9] 24 [JOB 10] flushing 2 memtables to L0 96 // I211213 16:23:49.134464 21136 3@vendor/github.com/cockroachdb/pebble/event.go:603 ⋮ [n9,pebble,s9] 26 [JOB 10] flushed 2 memtables to L0 [1535806] (1.3 M), in 0.2s, output rate 5.8 M/s 97 // current: 98 // I211213 16:23:48.903751 21136 99 // 3@vendor/github.com/cockroachdb/pebble/event.go:599 ⋮ [n9,pebble,s9] 24 [JOB 10] flushing 2 memtables (1.4MB) to L0 100 // I211213 16:23:49.134464 21136 101 // 3@vendor/github.com/cockroachdb/pebble/event.go:603 ⋮ [n9,pebble,s9] 26 [JOB 10] flushed 2 memtables (1.4MB) to L0 [1535806] (1.3MB), in 0.2s, output rate 5.8MB/s 102 // 103 // NOTE: we use the log timestamp to compute the flush duration rather than 104 // the Pebble log output. 105 flushPattern = regexp.MustCompile( 106 `^..*` + 107 /* Job ID */ `\[JOB (?P<job>\d+)]\s` + 108 /* Compaction type */ `flush(?P<suffix>ed|ing)\s` + 109 /* Memtable count; size (23.2+) */ `\d+ memtables? (\([^)]+\))?` + 110 /* SSTable Bytes */ `(?:.*?\((?P<bytes>[0-9.]+( [BKMGTPE]|[KMGTPE]?B))\))?`, 111 ) 112 flushPatternSuffixIdx = flushPattern.SubexpIndex("suffix") 113 flushPatternJobIdx = flushPattern.SubexpIndex("job") 114 flushPatternBytesIdx = flushPattern.SubexpIndex("bytes") 115 116 // Example ingested log lines: 117 // 23.1 and older: 118 // I220228 16:01:22.487906 18476248525 3@vendor/github.com/cockroachdb/pebble/ingest.go:637 ⋮ [n24,pebble,s24] 33430782 [JOB 10211226] ingested L0:21818678 (1.8 K), L0:21818683 (1.2 K), L0:21818679 (1.6 K), L0:21818680 (1.1 K), L0:21818681 (1.1 K), L0:21818682 (160 M) 119 // current: 120 // I220228 16:01:22.487906 18476248525 3@vendor/github.com/cockroachdb/pebble/ingest.go:637 ⋮ [n24,pebble,s24] 33430782 [JOB 10211226] ingested L0:21818678 (1.8KB), L0:21818683 (1.2KB), L0:21818679 (1.6KB), L0:21818680 (1.1KB), L0:21818681 (1.1KB), L0:21818682 (160MB) 121 // 122 ingestedPattern = regexp.MustCompile( 123 `^.*` + 124 /* Job ID */ `\[JOB (?P<job>\d+)]\s` + 125 /* ingested */ `ingested\s`) 126 ingestedPatternJobIdx = ingestedPattern.SubexpIndex("job") 127 ingestedFilePattern = regexp.MustCompile( 128 `L` + 129 /* Level */ `(?P<level>\d):` + 130 /* File number */ `(?P<file>\d+)\s` + 131 /* Bytes */ `\((?P<bytes>[0-9.]+( [BKMGTPE]|[KMGTPE]?B))\)`) 132 ingestedFilePatternLevelIdx = ingestedFilePattern.SubexpIndex("level") 133 ingestedFilePatternFileIdx = ingestedFilePattern.SubexpIndex("file") 134 ingestedFilePatternBytesIdx = ingestedFilePattern.SubexpIndex("bytes") 135 136 // flushable ingestions 137 // 138 // I230831 04:13:28.824280 3780 3@pebble/event.go:685 ⋮ [n10,s10,pebble] 365 [JOB 226] flushed 6 ingested flushables L0:024334 (1.5KB) + L0:024339 (1.0KB) + L0:024335 (1.9KB) + L0:024336 (1.1KB) + L0:024337 (1.1KB) + L0:024338 (12KB) in 0.0s (0.0s total), output rate 67MB/s 139 flushableIngestedPattern = regexp.MustCompile( 140 `^.*` + 141 /* Job ID */ `\[JOB (?P<job>\d+)]\s` + 142 /* match ingested flushable */ `flushed \d ingested flushable`) 143 flushableIngestedPatternJobIdx = flushableIngestedPattern.SubexpIndex("job") 144 145 // Example read-amp log line: 146 // 23.1 and older: 147 // total 31766 188 G - 257 G 187 G 48 K 3.6 G 744 536 G 49 K 278 G 5 2.1 148 // current: 149 // total | 1 639B 0B | - | 84B | 0 0B | 0 0B | 3 1.9KB | 1.2KB | 1 23.7 150 readAmpPattern = regexp.MustCompile( 151 /* Read amp */ `(?:^|\+)(?:\s{2}total|total \|).*?\s(?P<value>\d+)\s.{4,7}$`, 152 ) 153 readAmpPatternValueIdx = readAmpPattern.SubexpIndex("value") 154 ) 155 156 const ( 157 // timeFmt matches the Cockroach log timestamp format. 158 // See: https://github.com/cockroachdb/cockroach/blob/master/pkg/util/log/format_crdb_v2.go 159 timeFmt = "060102 15:04:05.000000" 160 161 // timeFmtSlim is similar to timeFmt, except that it strips components with a 162 // lower granularity than a minute. 163 timeFmtSlim = "060102 15:04" 164 165 // timeFmtHrMinSec prints only the hour, minute and second of the time. 166 timeFmtHrMinSec = "15:04:05" 167 ) 168 169 // compactionType is the type of compaction. It tracks the types in 170 // compaction.go. We copy the values here to avoid exporting the types in 171 // compaction.go. 172 type compactionType uint8 173 174 const ( 175 compactionTypeDefault compactionType = iota 176 compactionTypeFlush 177 compactionTypeMove 178 compactionTypeDeleteOnly 179 compactionTypeElisionOnly 180 compactionTypeRead 181 ) 182 183 // String implements fmt.Stringer. 184 func (c compactionType) String() string { 185 switch c { 186 case compactionTypeDefault: 187 return "default" 188 case compactionTypeMove: 189 return "move" 190 case compactionTypeDeleteOnly: 191 return "delete-only" 192 case compactionTypeElisionOnly: 193 return "elision-only" 194 case compactionTypeRead: 195 return "read" 196 default: 197 panic(errors.Newf("unknown compaction type: %s", c)) 198 } 199 } 200 201 // parseCompactionType parses the given compaction type string and returns a 202 // compactionType. 203 func parseCompactionType(s string) (t compactionType, err error) { 204 switch s { 205 case "default": 206 t = compactionTypeDefault 207 case "move": 208 t = compactionTypeMove 209 case "delete-only": 210 t = compactionTypeDeleteOnly 211 case "elision-only": 212 t = compactionTypeElisionOnly 213 case "read": 214 t = compactionTypeRead 215 default: 216 err = errors.Newf("unknown compaction type: %s", s) 217 } 218 return 219 } 220 221 // compactionStart is a compaction start event. 222 type compactionStart struct { 223 ctx logContext 224 jobID int 225 cType compactionType 226 fromLevel int 227 toLevel int 228 inputBytes uint64 229 } 230 231 // parseCompactionStart converts the given regular expression sub-matches for a 232 // compaction start log line into a compactionStart event. 233 func parseCompactionStart(matches []string) (compactionStart, error) { 234 var start compactionStart 235 236 // Parse job ID. 237 jobID, err := strconv.Atoi(matches[compactionPatternJobIdx]) 238 if err != nil { 239 return start, errors.Newf("could not parse jobID: %s", err) 240 } 241 242 // Parse compaction type. 243 cType, err := parseCompactionType(matches[compactionPatternTypeIdx]) 244 if err != nil { 245 return start, err 246 } 247 248 // Parse input bytes. 249 inputBytes, err := sumInputBytes(matches[compactionPatternLevels]) 250 if err != nil { 251 return start, errors.Newf("could not sum input bytes: %s", err) 252 } 253 254 // Parse from-level. 255 from, err := strconv.Atoi(matches[compactionPatternFromIdx]) 256 if err != nil { 257 return start, errors.Newf("could not parse from-level: %s", err) 258 } 259 260 // Parse to-level. For deletion and elision compactions, set the same level. 261 to := from 262 if cType != compactionTypeElisionOnly && cType != compactionTypeDeleteOnly { 263 to, err = strconv.Atoi(matches[compactionPatternToIdx]) 264 if err != nil { 265 return start, errors.Newf("could not parse to-level: %s", err) 266 } 267 } 268 269 start = compactionStart{ 270 jobID: jobID, 271 cType: cType, 272 fromLevel: from, 273 toLevel: to, 274 inputBytes: inputBytes, 275 } 276 277 return start, nil 278 } 279 280 // compactionEnd is a compaction end event. 281 type compactionEnd struct { 282 jobID int 283 writtenBytes uint64 284 // TODO(jackson): Parse and include the aggregate size of input 285 // sstables. It may be instructive, because compactions that drop 286 // keys write less data than they remove from the input level. 287 } 288 289 // parseCompactionEnd converts the given regular expression sub-matches for a 290 // compaction end log line into a compactionEnd event. 291 func parseCompactionEnd(matches []string) (compactionEnd, error) { 292 var end compactionEnd 293 294 // Parse job ID. 295 jobID, err := strconv.Atoi(matches[compactionPatternJobIdx]) 296 if err != nil { 297 return end, errors.Newf("could not parse jobID: %s", err) 298 } 299 end = compactionEnd{jobID: jobID} 300 301 // Optionally, if we have compacted bytes. 302 if matches[compactionPatternBytesIdx] != "" { 303 end.writtenBytes = unHumanize(matches[compactionPatternBytesIdx]) 304 } 305 306 return end, nil 307 } 308 309 // parseFlushStart converts the given regular expression sub-matches for a 310 // memtable flush start log line into a compactionStart event. 311 func parseFlushStart(matches []string) (compactionStart, error) { 312 var start compactionStart 313 // Parse job ID. 314 jobID, err := strconv.Atoi(matches[flushPatternJobIdx]) 315 if err != nil { 316 return start, errors.Newf("could not parse jobID: %s", err) 317 } 318 c := compactionStart{ 319 jobID: jobID, 320 cType: compactionTypeFlush, 321 fromLevel: -1, 322 toLevel: 0, 323 } 324 return c, nil 325 } 326 327 // parseFlushEnd converts the given regular expression sub-matches for a 328 // memtable flush end log line into a compactionEnd event. 329 func parseFlushEnd(matches []string) (compactionEnd, error) { 330 var end compactionEnd 331 332 // Parse job ID. 333 jobID, err := strconv.Atoi(matches[flushPatternJobIdx]) 334 if err != nil { 335 return end, errors.Newf("could not parse jobID: %s", err) 336 } 337 end = compactionEnd{jobID: jobID} 338 339 // Optionally, if we have flushed bytes. 340 if matches[flushPatternBytesIdx] != "" { 341 end.writtenBytes = unHumanize(matches[flushPatternBytesIdx]) 342 } 343 344 return end, nil 345 } 346 347 // event describes an aggregated event (eg, start and end events 348 // combined if necessary). 349 type event struct { 350 nodeID int 351 storeID int 352 jobID int 353 timeStart time.Time 354 timeEnd time.Time 355 compaction *compaction 356 ingest *ingest 357 } 358 359 // compaction represents an aggregated compaction event (i.e. the combination of 360 // a start and end event). 361 type compaction struct { 362 cType compactionType 363 fromLevel int 364 toLevel int 365 inputBytes uint64 366 outputBytes uint64 367 } 368 369 // ingest describes the completion of an ingest. 370 type ingest struct { 371 files []ingestedFile 372 } 373 374 type ingestedFile struct { 375 level int 376 fileNum int 377 sizeBytes uint64 378 } 379 380 // readAmp represents a read-amp event. 381 type readAmp struct { 382 ctx logContext 383 readAmp int 384 } 385 386 type nodeStoreJob struct { 387 node, store, job int 388 } 389 390 func (n nodeStoreJob) String() string { 391 return fmt.Sprintf("(node=%d,store=%d,job=%d)", n.node, n.store, n.job) 392 } 393 394 type errorEvent struct { 395 path string 396 line string 397 err error 398 } 399 400 // logEventCollector keeps track of open compaction events and read-amp events 401 // over the course of parsing log line events. Completed compaction events are 402 // added to the collector once a matching start and end pair are encountered. 403 // Read-amp events are added as they are encountered (the have no start / end 404 // concept). 405 type logEventCollector struct { 406 ctx logContext 407 m map[nodeStoreJob]compactionStart 408 events []event 409 readAmps []readAmp 410 errors []errorEvent 411 } 412 413 // newEventCollector instantiates a new logEventCollector. 414 func newEventCollector() *logEventCollector { 415 return &logEventCollector{ 416 m: make(map[nodeStoreJob]compactionStart), 417 } 418 } 419 420 // addError records an error encountered during log parsing. 421 func (c *logEventCollector) addError(path, line string, err error) { 422 c.errors = append(c.errors, errorEvent{path: path, line: line, err: err}) 423 } 424 425 // addCompactionStart adds a new compactionStart to the collector. The event is 426 // tracked by its job ID. 427 func (c *logEventCollector) addCompactionStart(start compactionStart) error { 428 key := nodeStoreJob{c.ctx.node, c.ctx.store, start.jobID} 429 if _, ok := c.m[key]; ok { 430 return errors.Newf("start event already seen for %s", key) 431 } 432 start.ctx = c.ctx 433 c.m[key] = start 434 return nil 435 } 436 437 // addCompactionEnd completes the compaction event for the given compactionEnd. 438 func (c *logEventCollector) addCompactionEnd(end compactionEnd) { 439 key := nodeStoreJob{c.ctx.node, c.ctx.store, end.jobID} 440 start, ok := c.m[key] 441 if !ok { 442 _, _ = fmt.Fprintf( 443 os.Stderr, 444 "compaction end event missing start event for %s; skipping\n", key, 445 ) 446 return 447 } 448 449 // Remove the job from the collector once it has been matched. 450 delete(c.m, key) 451 452 c.events = append(c.events, event{ 453 nodeID: start.ctx.node, 454 storeID: start.ctx.store, 455 jobID: start.jobID, 456 timeStart: start.ctx.timestamp, 457 timeEnd: c.ctx.timestamp, 458 compaction: &compaction{ 459 cType: start.cType, 460 fromLevel: start.fromLevel, 461 toLevel: start.toLevel, 462 inputBytes: start.inputBytes, 463 outputBytes: end.writtenBytes, 464 }, 465 }) 466 } 467 468 // addReadAmp adds the readAmp event to the collector. 469 func (c *logEventCollector) addReadAmp(ra readAmp) { 470 ra.ctx = c.ctx 471 c.readAmps = append(c.readAmps, ra) 472 } 473 474 // logContext captures the metadata of log lines. 475 type logContext struct { 476 timestamp time.Time 477 node, store int 478 } 479 480 // saveContext saves the given logContext in the collector. 481 func (c *logEventCollector) saveContext(ctx logContext) { 482 c.ctx = ctx 483 } 484 485 // level is a level in the LSM. The WAL is level -1. 486 type level int 487 488 // String implements fmt.Stringer. 489 func (l level) String() string { 490 if l == -1 { 491 return "WAL" 492 } 493 return "L" + strconv.Itoa(int(l)) 494 } 495 496 // fromTo is a map key for (from, to) level tuples. 497 type fromTo struct { 498 from, to level 499 } 500 501 // compactionTypeCount is a mapping from compaction type to count. 502 type compactionTypeCount map[compactionType]int 503 504 // windowSummary summarizes events in a window of time between a start and end 505 // time. The window tracks: 506 // - for each compaction type: counts, total bytes compacted, and total duration. 507 // - total ingested bytes for each level 508 // - read amp magnitudes 509 type windowSummary struct { 510 nodeID, storeID int 511 tStart, tEnd time.Time 512 eventCount int 513 flushedCount int 514 flushedBytes uint64 515 flushedTime time.Duration 516 compactionCounts map[fromTo]compactionTypeCount 517 compactionBytesIn map[fromTo]uint64 518 compactionBytesOut map[fromTo]uint64 519 compactionBytesMoved map[fromTo]uint64 520 compactionBytesDel map[fromTo]uint64 521 compactionTime map[fromTo]time.Duration 522 ingestedCount [numLevels]int 523 ingestedBytes [numLevels]uint64 524 readAmps []readAmp 525 longRunning []event 526 } 527 528 // String implements fmt.Stringer, returning a formatted window summary. 529 func (s windowSummary) String() string { 530 type fromToCount struct { 531 ft fromTo 532 counts compactionTypeCount 533 bytesIn uint64 534 bytesOut uint64 535 bytesMoved uint64 536 bytesDel uint64 537 duration time.Duration 538 } 539 var pairs []fromToCount 540 for k, v := range s.compactionCounts { 541 pairs = append(pairs, fromToCount{ 542 ft: k, 543 counts: v, 544 bytesIn: s.compactionBytesIn[k], 545 bytesOut: s.compactionBytesOut[k], 546 bytesMoved: s.compactionBytesMoved[k], 547 bytesDel: s.compactionBytesDel[k], 548 duration: s.compactionTime[k], 549 }) 550 } 551 slices.SortFunc(pairs, func(l, r fromToCount) int { 552 if v := cmp.Compare(l.ft.from, r.ft.from); v != 0 { 553 return v 554 } 555 return cmp.Compare(l.ft.to, r.ft.to) 556 }) 557 558 nodeID, storeID := "?", "?" 559 if s.nodeID != -1 { 560 nodeID = strconv.Itoa(s.nodeID) 561 } 562 if s.storeID != -1 { 563 storeID = strconv.Itoa(s.storeID) 564 } 565 566 var sb strings.Builder 567 sb.WriteString(fmt.Sprintf("node: %s, store: %s\n", nodeID, storeID)) 568 sb.WriteString(fmt.Sprintf(" from: %s\n", s.tStart.Format(timeFmtSlim))) 569 sb.WriteString(fmt.Sprintf(" to: %s\n", s.tEnd.Format(timeFmtSlim))) 570 var count, sum int 571 for _, ra := range s.readAmps { 572 count++ 573 sum += ra.readAmp 574 } 575 sb.WriteString(fmt.Sprintf(" r-amp: %.1f\n", float64(sum)/float64(count))) 576 577 // Print flush+ingest statistics. 578 { 579 var headerWritten bool 580 maybeWriteHeader := func() { 581 if !headerWritten { 582 sb.WriteString("_kind______from______to_____________________________________count___bytes______time\n") 583 headerWritten = true 584 } 585 } 586 587 if s.flushedCount > 0 { 588 maybeWriteHeader() 589 fmt.Fprintf(&sb, "%-7s %7s %7d %7s %9s\n", 590 "flush", "L0", s.flushedCount, humanize.Bytes.Uint64(s.flushedBytes), 591 s.flushedTime.Truncate(time.Second)) 592 } 593 594 count := s.flushedCount 595 sum := s.flushedBytes 596 totalTime := s.flushedTime 597 for l := 0; l < len(s.ingestedBytes); l++ { 598 if s.ingestedCount[l] == 0 { 599 continue 600 } 601 maybeWriteHeader() 602 fmt.Fprintf(&sb, "%-7s %7s %7d %7s\n", 603 "ingest", fmt.Sprintf("L%d", l), s.ingestedCount[l], humanize.Bytes.Uint64(s.ingestedBytes[l])) 604 count += s.ingestedCount[l] 605 sum += s.ingestedBytes[l] 606 } 607 if headerWritten { 608 fmt.Fprintf(&sb, "total %7d %7s %9s\n", 609 count, humanize.Bytes.Uint64(sum), totalTime.Truncate(time.Second), 610 ) 611 } 612 } 613 614 // Print compactions statistics. 615 if len(s.compactionCounts) > 0 { 616 sb.WriteString("_kind______from______to___default____move___elide__delete___count___in(B)__out(B)__mov(B)__del(B)______time\n") 617 var totalDef, totalMove, totalElision, totalDel int 618 var totalBytesIn, totalBytesOut, totalBytesMoved, totalBytesDel uint64 619 var totalTime time.Duration 620 for _, p := range pairs { 621 def := p.counts[compactionTypeDefault] 622 move := p.counts[compactionTypeMove] 623 elision := p.counts[compactionTypeElisionOnly] 624 del := p.counts[compactionTypeDeleteOnly] 625 total := def + move + elision + del 626 627 str := fmt.Sprintf("%-7s %7s %7s %7d %7d %7d %7d %7d %7s %7s %7s %7s %9s\n", 628 "compact", p.ft.from, p.ft.to, def, move, elision, del, total, 629 humanize.Bytes.Uint64(p.bytesIn), humanize.Bytes.Uint64(p.bytesOut), 630 humanize.Bytes.Uint64(p.bytesMoved), humanize.Bytes.Uint64(p.bytesDel), 631 p.duration.Truncate(time.Second)) 632 sb.WriteString(str) 633 634 totalDef += def 635 totalMove += move 636 totalElision += elision 637 totalDel += del 638 totalBytesIn += p.bytesIn 639 totalBytesOut += p.bytesOut 640 totalBytesMoved += p.bytesMoved 641 totalBytesDel += p.bytesDel 642 totalTime += p.duration 643 } 644 sb.WriteString(fmt.Sprintf("total %19d %7d %7d %7d %7d %7s %7s %7s %7s %9s\n", 645 totalDef, totalMove, totalElision, totalDel, s.eventCount, 646 humanize.Bytes.Uint64(totalBytesIn), humanize.Bytes.Uint64(totalBytesOut), 647 humanize.Bytes.Uint64(totalBytesMoved), humanize.Bytes.Uint64(totalBytesDel), 648 totalTime.Truncate(time.Second))) 649 } 650 651 // (Optional) Long running events. 652 if len(s.longRunning) > 0 { 653 sb.WriteString("long-running events (descending runtime):\n") 654 sb.WriteString("_kind________from________to_______job______type_____start_______end____dur(s)_____bytes:\n") 655 for _, e := range s.longRunning { 656 c := e.compaction 657 kind := "compact" 658 if c.fromLevel == -1 { 659 kind = "flush" 660 } 661 sb.WriteString(fmt.Sprintf("%-7s %9s %9s %9d %9s %9s %9s %9.0f %9s\n", 662 kind, level(c.fromLevel), level(c.toLevel), e.jobID, c.cType, 663 e.timeStart.Format(timeFmtHrMinSec), e.timeEnd.Format(timeFmtHrMinSec), 664 e.timeEnd.Sub(e.timeStart).Seconds(), humanize.Bytes.Uint64(c.outputBytes))) 665 } 666 } 667 668 return sb.String() 669 } 670 671 // windowSummarySlice is a slice of windowSummary that sorts in order of start 672 // time, node, then store. 673 type windowsSummarySlice []windowSummary 674 675 func (s windowsSummarySlice) Len() int { 676 return len(s) 677 } 678 679 func (s windowsSummarySlice) Less(i, j int) bool { 680 if !s[i].tStart.Equal(s[j].tStart) { 681 return s[i].tStart.Before(s[j].tStart) 682 } 683 if s[i].nodeID != s[j].nodeID { 684 return s[i].nodeID < s[j].nodeID 685 } 686 return s[i].storeID < s[j].storeID 687 } 688 689 func (s windowsSummarySlice) Swap(i, j int) { 690 s[i], s[j] = s[j], s[i] 691 } 692 693 // eventSlice is a slice of events that sorts in order of node, store, 694 // then event start time. 695 type eventSlice []event 696 697 func (s eventSlice) Len() int { 698 return len(s) 699 } 700 701 func (s eventSlice) Less(i, j int) bool { 702 if s[i].nodeID != s[j].nodeID { 703 return s[i].nodeID < s[j].nodeID 704 } 705 if s[i].storeID != s[j].storeID { 706 return s[i].storeID < s[j].storeID 707 } 708 return s[i].timeStart.Before(s[j].timeStart) 709 } 710 711 func (s eventSlice) Swap(i, j int) { 712 s[i], s[j] = s[j], s[i] 713 } 714 715 // readAmpSlice is a slice of readAmp events that sorts in order of node, store, 716 // then read amp event start time. 717 type readAmpSlice []readAmp 718 719 func (r readAmpSlice) Len() int { 720 return len(r) 721 } 722 723 func (r readAmpSlice) Less(i, j int) bool { 724 // Sort by node, store, then read-amp. 725 if r[i].ctx.node != r[j].ctx.node { 726 return r[i].ctx.node < r[j].ctx.node 727 } 728 if r[i].ctx.store != r[j].ctx.store { 729 return r[i].ctx.store < r[j].ctx.store 730 } 731 return r[i].ctx.timestamp.Before(r[j].ctx.timestamp) 732 } 733 734 func (r readAmpSlice) Swap(i, j int) { 735 r[i], r[j] = r[j], r[i] 736 } 737 738 // aggregator combines compaction and read-amp events within windows of fixed 739 // duration and returns one aggregated windowSummary struct per window. 740 type aggregator struct { 741 window time.Duration 742 events []event 743 readAmps []readAmp 744 longRunningLimit time.Duration 745 } 746 747 // newAggregator returns a new aggregator. 748 func newAggregator( 749 window, longRunningLimit time.Duration, events []event, readAmps []readAmp, 750 ) *aggregator { 751 return &aggregator{ 752 window: window, 753 events: events, 754 readAmps: readAmps, 755 longRunningLimit: longRunningLimit, 756 } 757 } 758 759 // aggregate aggregates the events into windows, returning the windowSummary for 760 // each interval. 761 func (a *aggregator) aggregate() []windowSummary { 762 if len(a.events) == 0 { 763 return nil 764 } 765 766 // Sort the event and read-amp slices by start time. 767 sort.Sort(eventSlice(a.events)) 768 sort.Sort(readAmpSlice(a.readAmps)) 769 770 initWindow := func(e event) *windowSummary { 771 start := e.timeStart.Truncate(a.window) 772 return &windowSummary{ 773 nodeID: e.nodeID, 774 storeID: e.storeID, 775 tStart: start, 776 tEnd: start.Add(a.window), 777 compactionCounts: make(map[fromTo]compactionTypeCount), 778 compactionBytesIn: make(map[fromTo]uint64), 779 compactionBytesOut: make(map[fromTo]uint64), 780 compactionBytesMoved: make(map[fromTo]uint64), 781 compactionBytesDel: make(map[fromTo]uint64), 782 compactionTime: make(map[fromTo]time.Duration), 783 } 784 } 785 786 var windows []windowSummary 787 var j int // index for read-amps 788 finishWindow := func(cur *windowSummary) { 789 // Collect read-amp values for the previous window. 790 var readAmps []readAmp 791 for j < len(a.readAmps) { 792 ra := a.readAmps[j] 793 794 // Skip values before the current window. 795 if ra.ctx.node < cur.nodeID || 796 ra.ctx.store < cur.storeID || 797 ra.ctx.timestamp.Before(cur.tStart) { 798 j++ 799 continue 800 } 801 802 // We've passed over the current window. Stop. 803 if ra.ctx.node > cur.nodeID || 804 ra.ctx.store > cur.storeID || 805 ra.ctx.timestamp.After(cur.tEnd) { 806 break 807 } 808 809 // Collect this read-amp value. 810 readAmps = append(readAmps, ra) 811 j++ 812 } 813 cur.readAmps = readAmps 814 815 // Sort long running compactions in descending order of duration. 816 slices.SortFunc(cur.longRunning, func(l, r event) int { 817 return cmp.Compare(l.timeEnd.Sub(l.timeStart), r.timeEnd.Sub(r.timeStart)) 818 }) 819 820 // Add the completed window to the set of windows. 821 windows = append(windows, *cur) 822 } 823 824 // Move through the compactions, collecting relevant compactions into the same 825 // window. Windows have the same node and store, and a compaction start time 826 // within a given range. 827 i := 0 828 curWindow := initWindow(a.events[i]) 829 for ; ; i++ { 830 // No more windows. Complete the current window. 831 if i == len(a.events) { 832 finishWindow(curWindow) 833 break 834 } 835 e := a.events[i] 836 837 // If we're at the start of a new interval, finalize the current window and 838 // start a new one. 839 if curWindow.nodeID != e.nodeID || 840 curWindow.storeID != e.storeID || 841 e.timeStart.After(curWindow.tEnd) { 842 finishWindow(curWindow) 843 curWindow = initWindow(e) 844 } 845 846 switch { 847 case e.ingest != nil: 848 // Update ingest stats. 849 for _, f := range e.ingest.files { 850 curWindow.ingestedCount[f.level]++ 851 curWindow.ingestedBytes[f.level] += f.sizeBytes 852 } 853 case e.compaction != nil && e.compaction.cType == compactionTypeFlush: 854 // Update flush stats. 855 f := e.compaction 856 curWindow.flushedCount++ 857 curWindow.flushedBytes += f.outputBytes 858 curWindow.flushedTime += e.timeEnd.Sub(e.timeStart) 859 case e.compaction != nil: 860 // Update compaction stats. 861 c := e.compaction 862 // Update compaction counts. 863 ft := fromTo{level(c.fromLevel), level(c.toLevel)} 864 m, ok := curWindow.compactionCounts[ft] 865 if !ok { 866 m = make(compactionTypeCount) 867 curWindow.compactionCounts[ft] = m 868 } 869 m[c.cType]++ 870 curWindow.eventCount++ 871 872 // Update compacted bytes in / out / moved / deleted. 873 switch c.cType { 874 case compactionTypeMove: 875 curWindow.compactionBytesMoved[ft] += c.inputBytes 876 case compactionTypeDeleteOnly: 877 curWindow.compactionBytesDel[ft] += c.inputBytes 878 default: 879 curWindow.compactionBytesIn[ft] += c.inputBytes 880 curWindow.compactionBytesOut[ft] += c.outputBytes 881 } 882 883 // Update compaction time. 884 _, ok = curWindow.compactionTime[ft] 885 if !ok { 886 curWindow.compactionTime[ft] = 0 887 } 888 curWindow.compactionTime[ft] += e.timeEnd.Sub(e.timeStart) 889 890 } 891 // Add "long-running" events. Those that start in this window 892 // that have duration longer than the window interval. 893 if e.timeEnd.Sub(e.timeStart) > a.longRunningLimit { 894 curWindow.longRunning = append(curWindow.longRunning, e) 895 } 896 } 897 898 // Windows are added in order of (node, store, time). Re-sort the windows by 899 // (time, node, store) for better presentation. 900 sort.Sort(windowsSummarySlice(windows)) 901 902 return windows 903 } 904 905 // parseLog parses the log file with the given path, using the given parse 906 // function to collect events in the given logEventCollector. parseLog 907 // returns a non-nil error if an I/O error was encountered while reading 908 // the log file. Parsing errors are accumulated in the 909 // logEventCollector. 910 func parseLog(path string, b *logEventCollector) error { 911 f, err := os.Open(path) 912 if err != nil { 913 return err 914 } 915 defer f.Close() 916 917 s := bufio.NewScanner(f) 918 for s.Scan() { 919 line := s.Text() 920 // Store the log context for the current line, if we have one. 921 if err := parseLogContext(line, b); err != nil { 922 return err 923 } 924 925 // First check for a flush or compaction. 926 matches := sentinelPattern.FindStringSubmatch(line) 927 if matches != nil { 928 // Determine which regexp to apply by testing the first letter of the prefix. 929 var err error 930 switch matches[sentinelPatternPrefixIdx][0] { 931 case 'c': 932 err = parseCompaction(line, b) 933 case 'f': 934 err = parseFlush(line, b) 935 case 'i': 936 err = parseIngest(line, b) 937 default: 938 err = errors.Newf("unexpected line: neither compaction nor flush: %s", line) 939 } 940 if err != nil { 941 b.addError(path, line, err) 942 } 943 continue 944 } 945 946 // Else check for an LSM debug line. 947 if err = parseReadAmp(line, b); err != nil { 948 b.addError(path, line, err) 949 continue 950 } 951 } 952 return s.Err() 953 } 954 955 // parseLogContext extracts contextual information from the log line (e.g. the 956 // timestamp, node and store). 957 func parseLogContext(line string, b *logEventCollector) error { 958 matches := logContextPattern.FindStringSubmatch(line) 959 if matches == nil { 960 return nil 961 } 962 963 // Parse start time. 964 t, err := time.Parse(timeFmt, matches[logContextPatternTimestampIdx]) 965 if err != nil { 966 return errors.Newf("could not parse timestamp: %s", err) 967 } 968 969 // Parse node and store. 970 nodeID, err := strconv.Atoi(matches[logContextPatternNodeIdx]) 971 if err != nil { 972 if matches[logContextPatternNodeIdx] != "?" { 973 return errors.Newf("could not parse node ID: %s", err) 974 } 975 nodeID = -1 976 } 977 978 storeID, err := strconv.Atoi(matches[logContextPatternStoreIdx]) 979 if err != nil { 980 if matches[logContextPatternStoreIdx] != "?" { 981 return errors.Newf("could not parse store ID: %s", err) 982 } 983 storeID = -1 984 } 985 986 b.saveContext(logContext{ 987 timestamp: t, 988 node: nodeID, 989 store: storeID, 990 }) 991 return nil 992 } 993 994 // parseCompaction parses and collects Pebble compaction events. 995 func parseCompaction(line string, b *logEventCollector) error { 996 matches := compactionPattern.FindStringSubmatch(line) 997 if matches == nil { 998 return nil 999 } 1000 1001 // "compacting": implies start line. 1002 if matches[compactionPatternSuffixIdx] == "ing" { 1003 start, err := parseCompactionStart(matches) 1004 if err != nil { 1005 return err 1006 } 1007 if err := b.addCompactionStart(start); err != nil { 1008 return err 1009 } 1010 return nil 1011 } 1012 1013 // "compacted": implies end line. 1014 end, err := parseCompactionEnd(matches) 1015 if err != nil { 1016 return err 1017 } 1018 1019 b.addCompactionEnd(end) 1020 return nil 1021 } 1022 1023 // parseFlush parses and collects Pebble memtable flush events. 1024 func parseFlush(line string, b *logEventCollector) error { 1025 matches := flushPattern.FindStringSubmatch(line) 1026 if matches == nil { 1027 return nil 1028 } 1029 1030 if matches[flushPatternSuffixIdx] == "ing" { 1031 start, err := parseFlushStart(matches) 1032 if err != nil { 1033 return err 1034 } 1035 return b.addCompactionStart(start) 1036 } 1037 1038 end, err := parseFlushEnd(matches) 1039 if err != nil { 1040 return err 1041 } 1042 1043 b.addCompactionEnd(end) 1044 return nil 1045 } 1046 1047 func parseIngestDuringFlush(line string, b *logEventCollector) error { 1048 matches := flushableIngestedPattern.FindStringSubmatch(line) 1049 if matches == nil { 1050 return nil 1051 } 1052 // Parse job ID. 1053 jobID, err := strconv.Atoi(matches[flushableIngestedPatternJobIdx]) 1054 if err != nil { 1055 return errors.Newf("could not parse jobID: %s", err) 1056 } 1057 return parseRemainingIngestLogLine(jobID, line, b) 1058 } 1059 1060 // parseIngest parses and collects Pebble ingest complete events. 1061 func parseIngest(line string, b *logEventCollector) error { 1062 matches := ingestedPattern.FindStringSubmatch(line) 1063 if matches == nil { 1064 // Try and parse the other kind of ingest. 1065 return parseIngestDuringFlush(line, b) 1066 } 1067 // Parse job ID. 1068 jobID, err := strconv.Atoi(matches[ingestedPatternJobIdx]) 1069 if err != nil { 1070 return errors.Newf("could not parse jobID: %s", err) 1071 } 1072 return parseRemainingIngestLogLine(jobID, line, b) 1073 } 1074 1075 // parses the level, filenum, and bytes for the files which were ingested. 1076 func parseRemainingIngestLogLine(jobID int, line string, b *logEventCollector) error { 1077 fileMatches := ingestedFilePattern.FindAllStringSubmatch(line, -1) 1078 files := make([]ingestedFile, len(fileMatches)) 1079 for i := range fileMatches { 1080 level, err := strconv.Atoi(fileMatches[i][ingestedFilePatternLevelIdx]) 1081 if err != nil { 1082 return errors.Newf("could not parse level: %s", err) 1083 } 1084 fileNum, err := strconv.Atoi(fileMatches[i][ingestedFilePatternFileIdx]) 1085 if err != nil { 1086 return errors.Newf("could not parse file number: %s", err) 1087 } 1088 files[i] = ingestedFile{ 1089 level: level, 1090 fileNum: fileNum, 1091 sizeBytes: unHumanize(fileMatches[i][ingestedFilePatternBytesIdx]), 1092 } 1093 } 1094 b.events = append(b.events, event{ 1095 nodeID: b.ctx.node, 1096 storeID: b.ctx.store, 1097 jobID: jobID, 1098 timeStart: b.ctx.timestamp, 1099 timeEnd: b.ctx.timestamp, 1100 ingest: &ingest{ 1101 files: files, 1102 }, 1103 }) 1104 return nil 1105 } 1106 1107 // parseReadAmp attempts to parse the current line as a read amp value 1108 func parseReadAmp(line string, b *logEventCollector) error { 1109 matches := readAmpPattern.FindStringSubmatch(line) 1110 if matches == nil { 1111 return nil 1112 } 1113 val, err := strconv.Atoi(matches[readAmpPatternValueIdx]) 1114 if err != nil { 1115 return errors.Newf("could not parse read amp: %s", err) 1116 } 1117 b.addReadAmp(readAmp{ 1118 readAmp: val, 1119 }) 1120 return nil 1121 } 1122 1123 // runCompactionLogs is runnable function of the top-level cobra.Command that 1124 // parses and collects Pebble compaction events and LSM information. 1125 func runCompactionLogs(cmd *cobra.Command, args []string) error { 1126 // The args contain a list of log files to read. 1127 files := args 1128 1129 // Scan the log files collecting start and end compaction lines. 1130 b := newEventCollector() 1131 for _, file := range files { 1132 err := parseLog(file, b) 1133 // parseLog returns an error only on I/O errors, which we 1134 // immediately exit with. 1135 if err != nil { 1136 return err 1137 } 1138 } 1139 1140 window, err := cmd.Flags().GetDuration("window") 1141 if err != nil { 1142 return err 1143 } 1144 1145 longRunningLimit, err := cmd.Flags().GetDuration("long-running-limit") 1146 if err != nil { 1147 return err 1148 } 1149 if longRunningLimit == 0 { 1150 // Off by default. Set to infinite duration. 1151 longRunningLimit = time.Duration(math.MaxInt64) 1152 } 1153 1154 // Aggregate the lines. 1155 a := newAggregator(window, longRunningLimit, b.events, b.readAmps) 1156 summaries := a.aggregate() 1157 for _, s := range summaries { 1158 fmt.Printf("%s\n", s) 1159 } 1160 1161 // After the summaries, print accumulated parsing errors to stderr. 1162 for _, e := range b.errors { 1163 fmt.Fprintf(os.Stderr, "-\n%s: %s\nError: %s\n", filepath.Base(e.path), e.line, e.err) 1164 } 1165 return nil 1166 } 1167 1168 // unHumanize performs the opposite of humanize.Bytes.Uint64 (e.g. "10B", 1169 // "10MB") or the 23.1 humanize.IEC.Uint64 (e.g. "10 B", "10 M"), converting a 1170 // human-readable value into a raw number of bytes. 1171 func unHumanize(s string) uint64 { 1172 if len(s) < 2 || !(s[0] >= '0' && s[0] <= '9') { 1173 panic(errors.Newf("invalid bytes value %q", s)) 1174 } 1175 if s[len(s)-1] == 'B' { 1176 s = s[:len(s)-1] 1177 } 1178 1179 multiplier := uint64(1) 1180 switch s[len(s)-1] { 1181 case 'K': 1182 multiplier = 1 << 10 1183 case 'M': 1184 multiplier = 1 << 20 1185 case 'G': 1186 multiplier = 1 << 30 1187 case 'T': 1188 multiplier = 1 << 40 1189 case 'P': 1190 multiplier = 1 << 50 1191 case 'E': 1192 multiplier = 1 << 60 1193 } 1194 if multiplier != 1 { 1195 s = s[:len(s)-1] 1196 } 1197 if s[len(s)-1] == ' ' { 1198 s = s[:len(s)-1] 1199 } 1200 val, err := strconv.ParseFloat(s, 64) 1201 if err != nil { 1202 panic(fmt.Sprintf("parsing %s: %v", s, err)) 1203 } 1204 1205 return uint64(val * float64(multiplier)) 1206 } 1207 1208 // sumInputBytes takes a string as input and returns the sum of the 1209 // human-readable sizes, as an integer number of bytes. 1210 func sumInputBytes(s string) (total uint64, _ error) { 1211 var ( 1212 open bool 1213 b bytes.Buffer 1214 ) 1215 for _, c := range s { 1216 switch c { 1217 case '(': 1218 open = true 1219 case ')': 1220 total += unHumanize(b.String()) 1221 b.Reset() 1222 open = false 1223 default: 1224 if open { 1225 b.WriteRune(c) 1226 } 1227 } 1228 } 1229 return 1230 }