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