github.com/cockroachdb/pebble@v1.1.2/internal/pacertoy/rocksdb/main.go (about) 1 // Copyright 2019 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 main 6 7 import ( 8 "fmt" 9 "math" 10 "sync" 11 "sync/atomic" 12 "time" 13 14 "github.com/cockroachdb/pebble/internal/rate" 15 "golang.org/x/exp/rand" 16 ) 17 18 const ( 19 // Max rate for all compactions. This is intentionally set low enough that 20 // user writes will have to be delayed. 21 maxCompactionRate = 80 << 20 // 80 MB/s 22 23 memtableSize = 64 << 20 // 64 MB 24 memtableStopThreshold = 2 * memtableSize 25 maxWriteRate = 30 << 20 // 30 MB/s 26 startingWriteRate = 30 << 20 // 30 MB/s 27 28 l0SlowdownThreshold = 4 29 l0CompactionThreshold = 1 30 31 levelRatio = 10 32 numLevels = 7 33 34 // Slowdown threshold is set at the compaction debt incurred by the largest 35 // possible compaction. 36 compactionDebtSlowdownThreshold = memtableSize * (numLevels - 2) 37 ) 38 39 type compactionPacer struct { 40 level atomic.Int64 41 drainer *rate.Limiter 42 } 43 44 func newCompactionPacer() *compactionPacer { 45 p := &compactionPacer{ 46 drainer: rate.NewLimiter(maxCompactionRate, maxCompactionRate), 47 } 48 return p 49 } 50 51 func (p *compactionPacer) fill(n int64) { 52 p.level.Add(n) 53 } 54 55 func (p *compactionPacer) drain(n int64) { 56 p.drainer.Wait(float64(n)) 57 58 p.level.Add(-n) 59 } 60 61 type flushPacer struct { 62 level atomic.Int64 63 memtableStopThreshold float64 64 fillCond sync.Cond 65 } 66 67 func newFlushPacer(mu *sync.Mutex) *flushPacer { 68 p := &flushPacer{ 69 memtableStopThreshold: memtableStopThreshold, 70 } 71 p.fillCond.L = mu 72 return p 73 } 74 75 func (p *flushPacer) fill(n int64) { 76 for float64(p.level.Load()) >= p.memtableStopThreshold { 77 p.fillCond.Wait() 78 } 79 p.level.Add(n) 80 p.fillCond.Signal() 81 } 82 83 func (p *flushPacer) drain(n int64) { 84 p.level.Add(-n) 85 } 86 87 // DB models a RocksDB DB. 88 type DB struct { 89 mu sync.Mutex 90 flushPacer *flushPacer 91 flushCond sync.Cond 92 memtables []*int64 93 fill atomic.Int64 94 drain atomic.Int64 95 96 compactionMu sync.Mutex 97 compactionPacer *compactionPacer 98 // L0 is represented as an array of integers whereas every other level 99 // is represented as a single integer. 100 L0 []*int64 101 // Non-L0 sstables. sstables[0] == L1. 102 sstables []atomic.Int64 103 maxSSTableSizes []int64 104 compactionFlushCond sync.Cond 105 prevCompactionDebt float64 106 previouslyInDebt bool 107 108 writeLimiter *rate.Limiter 109 } 110 111 func newDB() *DB { 112 db := &DB{} 113 db.flushPacer = newFlushPacer(&db.mu) 114 db.flushCond.L = &db.mu 115 db.memtables = append(db.memtables, new(int64)) 116 117 db.compactionFlushCond.L = &db.compactionMu 118 db.L0 = append(db.L0, new(int64)) 119 db.compactionPacer = newCompactionPacer() 120 121 db.maxSSTableSizes = make([]int64, numLevels-1) 122 db.sstables = make([]atomic.Int64, numLevels-1) 123 base := int64(levelRatio) 124 for i := uint64(0); i < numLevels-2; i++ { 125 // Each level is 10 times larger than the one above it. 126 db.maxSSTableSizes[i] = memtableSize * l0CompactionThreshold * base 127 base *= levelRatio 128 129 // Begin with each level full. 130 newLevel := db.maxSSTableSizes[i] 131 132 db.sstables[i].Store(newLevel) 133 } 134 db.sstables[numLevels-2].Store(0) 135 db.maxSSTableSizes[numLevels-2] = math.MaxInt64 136 137 db.writeLimiter = rate.NewLimiter(startingWriteRate, startingWriteRate) 138 139 go db.drainMemtable() 140 go db.drainCompaction() 141 142 return db 143 } 144 145 // drainCompaction simulates background compactions. 146 func (db *DB) drainCompaction() { 147 rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) 148 149 for { 150 db.compactionMu.Lock() 151 152 for len(db.L0) <= l0CompactionThreshold { 153 db.compactionFlushCond.Wait() 154 } 155 l0Table := db.L0[0] 156 db.compactionMu.Unlock() 157 158 for i, size := int64(0), int64(0); i < *l0Table; i += size { 159 size = 10000 + rng.Int63n(500) 160 if size > (*l0Table - i) { 161 size = *l0Table - i 162 } 163 db.compactionPacer.drain(size) 164 } 165 166 db.compactionMu.Lock() 167 db.L0 = db.L0[1:] 168 db.compactionMu.Unlock() 169 170 singleTableSize := int64(memtableSize) 171 tablesToCompact := 0 172 for i := range db.sstables { 173 newSSTableSize := db.sstables[i].Add(singleTableSize) 174 if newSSTableSize > db.maxSSTableSizes[i] { 175 db.sstables[i].Add(-singleTableSize) 176 tablesToCompact++ 177 } else { 178 // Lower levels do not need compaction if level above it did not 179 // need compaction. 180 break 181 } 182 } 183 184 totalCompactionBytes := int64(tablesToCompact * memtableSize) 185 db.compactionPacer.fill(totalCompactionBytes) 186 187 for t := 0; t < tablesToCompact; t++ { 188 for i, size := int64(0), int64(0); i < memtableSize; i += size { 189 size = 10000 + rng.Int63n(500) 190 if size > (totalCompactionBytes - i) { 191 size = totalCompactionBytes - i 192 } 193 db.compactionPacer.drain(size) 194 } 195 196 db.delayUserWrites() 197 } 198 } 199 } 200 201 // fillCompaction fills L0 sstables. 202 func (db *DB) fillCompaction(size int64) { 203 db.compactionMu.Lock() 204 205 db.compactionPacer.fill(size) 206 207 last := db.L0[len(db.L0)-1] 208 if *last+size > memtableSize { 209 last = new(int64) 210 db.L0 = append(db.L0, last) 211 db.compactionFlushCond.Signal() 212 } 213 *last += size 214 215 db.compactionMu.Unlock() 216 } 217 218 // drainMemtable simulates memtable flushing. 219 func (db *DB) drainMemtable() { 220 rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) 221 222 for { 223 db.mu.Lock() 224 for len(db.memtables) <= 1 { 225 db.flushCond.Wait() 226 } 227 memtable := db.memtables[0] 228 db.mu.Unlock() 229 230 for i, size := int64(0), int64(0); i < *memtable; i += size { 231 size = 1000 + rng.Int63n(50) 232 if size > (*memtable - i) { 233 size = *memtable - i 234 } 235 db.flushPacer.drain(size) 236 db.drain.Add(size) 237 238 db.fillCompaction(size) 239 } 240 241 db.delayUserWrites() 242 243 db.mu.Lock() 244 db.memtables = db.memtables[1:] 245 db.mu.Unlock() 246 } 247 } 248 249 // delayUserWrites applies write delays depending on compaction debt. 250 func (db *DB) delayUserWrites() { 251 totalCompactionBytes := db.compactionPacer.level.Load() 252 compactionDebt := math.Max(float64(totalCompactionBytes)-l0CompactionThreshold*memtableSize, 0.0) 253 254 db.mu.Lock() 255 if len(db.L0) > l0SlowdownThreshold || compactionDebt > compactionDebtSlowdownThreshold { 256 db.previouslyInDebt = true 257 if compactionDebt > db.prevCompactionDebt { 258 // Debt is growing. 259 drainLimit := db.writeLimiter.Rate() * 0.8 260 if drainLimit > 0 { 261 db.writeLimiter.SetRate(drainLimit) 262 } 263 } else { 264 // Debt is shrinking. 265 drainLimit := db.writeLimiter.Rate() * 1 / 0.8 266 if drainLimit <= maxWriteRate { 267 db.writeLimiter.SetRate(drainLimit) 268 } 269 } 270 } else if db.previouslyInDebt { 271 // If compaction was previously delayed and has recovered, RocksDB 272 // "rewards" the rate by double the slowdown ratio. 273 274 // From RocksDB: 275 // If the DB recovers from delay conditions, we reward with reducing 276 // double the slowdown ratio. This is to balance the long term slowdown 277 // increase signal. 278 drainLimit := db.writeLimiter.Rate() * 1.4 279 if drainLimit <= maxWriteRate { 280 db.writeLimiter.SetRate(drainLimit) 281 } 282 db.previouslyInDebt = false 283 } 284 285 db.prevCompactionDebt = compactionDebt 286 db.mu.Unlock() 287 } 288 289 // fillMemtable simulates memtable filling. 290 func (db *DB) fillMemtable(size int64) { 291 db.mu.Lock() 292 293 db.flushPacer.fill(size) 294 db.fill.Add(size) 295 296 last := db.memtables[len(db.memtables)-1] 297 if *last+size > memtableSize { 298 last = new(int64) 299 db.memtables = append(db.memtables, last) 300 db.flushCond.Signal() 301 } 302 *last += size 303 304 db.mu.Unlock() 305 } 306 307 // simulateWrite simulates user writes. 308 func simulateWrite(db *DB) { 309 limiter := rate.NewLimiter(10<<20, 10<<20) // 10 MB/s 310 fmt.Printf("filling at 10 MB/sec\n") 311 312 setRate := func(mb int) { 313 fmt.Printf("filling at %d MB/sec\n", mb) 314 limiter.SetRate(float64(mb << 20)) 315 } 316 317 go func() { 318 rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) 319 for { 320 secs := 5 + rng.Intn(5) 321 time.Sleep(time.Duration(secs) * time.Second) 322 mb := 11 + rng.Intn(20) 323 setRate(mb) 324 } 325 }() 326 327 rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) 328 329 for { 330 size := 1000 + rng.Int63n(50) 331 limiter.Wait(float64(size)) 332 db.writeLimiter.Wait(float64(size)) 333 db.fillMemtable(size) 334 } 335 } 336 337 func main() { 338 db := newDB() 339 340 go simulateWrite(db) 341 342 tick := time.NewTicker(time.Second) 343 start := time.Now() 344 lastNow := start 345 var lastFill, lastDrain int64 346 347 for i := 0; ; i++ { 348 <-tick.C 349 if (i % 20) == 0 { 350 fmt.Printf("_elapsed___memtbs____dirty_____fill____drain____cdebt__l0count___max-w-rate\n") 351 } 352 353 db.mu.Lock() 354 memtableCount := len(db.memtables) 355 db.mu.Unlock() 356 dirty := db.flushPacer.level.Load() 357 fill := db.fill.Load() 358 drain := db.drain.Load() 359 360 db.compactionMu.Lock() 361 compactionL0 := len(db.L0) 362 db.compactionMu.Unlock() 363 totalCompactionBytes := db.compactionPacer.level.Load() 364 compactionDebt := math.Max(float64(totalCompactionBytes)-l0CompactionThreshold*memtableSize, 0.0) 365 maxWriteRate := db.writeLimiter.Rate() 366 367 now := time.Now() 368 elapsed := now.Sub(lastNow).Seconds() 369 fmt.Printf("%8s %8d %8.1f %8.1f %8.1f %8.1f %8d %12.1f\n", 370 time.Duration(now.Sub(start).Seconds()+0.5)*time.Second, 371 memtableCount, 372 float64(dirty)/(1024.0*1024.0), 373 float64(fill-lastFill)/(1024.0*1024.0*elapsed), 374 float64(drain-lastDrain)/(1024.0*1024.0*elapsed), 375 compactionDebt/(1024.0*1024.0), 376 compactionL0, 377 maxWriteRate/(1024.0*1024.0)) 378 379 lastNow = now 380 lastFill = fill 381 lastDrain = drain 382 } 383 }