github.com/zuoyebang/bitalostable@v1.0.1-0.20240229032404-e3b99a834294/db.go (about) 1 // Copyright 2012 The LevelDB-Go and Pebble Authors and Bitalostored 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 bitalostable 6 7 import ( 8 "fmt" 9 "io" 10 "os" 11 "sync" 12 "sync/atomic" 13 "time" 14 15 "github.com/cockroachdb/errors" 16 "github.com/zuoyebang/bitalostable/internal/arenaskl" 17 "github.com/zuoyebang/bitalostable/internal/base" 18 "github.com/zuoyebang/bitalostable/internal/invariants" 19 "github.com/zuoyebang/bitalostable/internal/keyspan" 20 "github.com/zuoyebang/bitalostable/internal/manifest" 21 "github.com/zuoyebang/bitalostable/internal/manual" 22 "github.com/zuoyebang/bitalostable/record" 23 "github.com/zuoyebang/bitalostable/sstable" 24 "github.com/zuoyebang/bitalostable/vfs" 25 "github.com/zuoyebang/bitalostable/vfs/atomicfs" 26 ) 27 28 const ( 29 // minTableCacheSize is the minimum size of the table cache, for a single db. 30 minTableCacheSize = 64 31 32 // numNonTableCacheFiles is an approximation for the number of files 33 // that we don't use for table caches, for a given db. 34 numNonTableCacheFiles = 10 35 ) 36 37 var ( 38 // ErrNotFound is returned when a get operation does not find the requested 39 // key. 40 ErrNotFound = base.ErrNotFound 41 // ErrClosed is panicked when an operation is performed on a closed snapshot or 42 // DB. Use errors.Is(err, ErrClosed) to check for this error. 43 ErrClosed = errors.New("bitalostable: closed") 44 // ErrReadOnly is returned when a write operation is performed on a read-only 45 // database. 46 ErrReadOnly = errors.New("bitalostable: read-only") 47 // errNoSplit indicates that the user is trying to perform a range key 48 // operation but the configured Comparer does not provide a Split 49 // implementation. 50 errNoSplit = errors.New("bitalostable: Comparer.Split required for range key operations") 51 ) 52 53 // Reader is a readable key/value store. 54 // 55 // It is safe to call Get and NewIter from concurrent goroutines. 56 type Reader interface { 57 // Get gets the value for the given key. It returns ErrNotFound if the DB 58 // does not contain the key. 59 // 60 // The caller should not modify the contents of the returned slice, but it is 61 // safe to modify the contents of the argument after Get returns. The 62 // returned slice will remain valid until the returned Closer is closed. On 63 // success, the caller MUST call closer.Close() or a memory leak will occur. 64 Get(key []byte) (value []byte, closer io.Closer, err error) 65 66 // NewIter returns an iterator that is unpositioned (Iterator.Valid() will 67 // return false). The iterator can be positioned via a call to SeekGE, 68 // SeekLT, First or Last. 69 NewIter(o *IterOptions) *Iterator 70 71 // Close closes the Reader. It may or may not close any underlying io.Reader 72 // or io.Writer, depending on how the DB was created. 73 // 74 // It is not safe to close a DB until all outstanding iterators are closed. 75 // It is valid to call Close multiple times. Other methods should not be 76 // called after the DB has been closed. 77 Close() error 78 } 79 80 // Writer is a writable key/value store. 81 // 82 // Goroutine safety is dependent on the specific implementation. 83 type Writer interface { 84 // Apply the operations contained in the batch to the DB. 85 // 86 // It is safe to modify the contents of the arguments after Apply returns. 87 Apply(batch *Batch, o *WriteOptions) error 88 89 // Delete deletes the value for the given key. Deletes are blind all will 90 // succeed even if the given key does not exist. 91 // 92 // It is safe to modify the contents of the arguments after Delete returns. 93 Delete(key []byte, o *WriteOptions) error 94 95 // SingleDelete is similar to Delete in that it deletes the value for the given key. Like Delete, 96 // it is a blind operation that will succeed even if the given key does not exist. 97 // 98 // WARNING: Undefined (non-deterministic) behavior will result if a key is overwritten and 99 // then deleted using SingleDelete. The record may appear deleted immediately, but be 100 // resurrected at a later time after compactions have been performed. Or the record may 101 // be deleted permanently. A Delete operation lays down a "tombstone" which shadows all 102 // previous versions of a key. The SingleDelete operation is akin to "anti-matter" and will 103 // only delete the most recently written version for a key. These different semantics allow 104 // the DB to avoid propagating a SingleDelete operation during a compaction as soon as the 105 // corresponding Set operation is encountered. These semantics require extreme care to handle 106 // properly. Only use if you have a workload where the performance gain is critical and you 107 // can guarantee that a record is written once and then deleted once. 108 // 109 // SingleDelete is internally transformed into a Delete if the most recent record for a key is either 110 // a Merge or Delete record. 111 // 112 // It is safe to modify the contents of the arguments after SingleDelete returns. 113 SingleDelete(key []byte, o *WriteOptions) error 114 115 // DeleteRange deletes all of the point keys (and values) in the range 116 // [start,end) (inclusive on start, exclusive on end). DeleteRange does NOT 117 // delete overlapping range keys (eg, keys set via RangeKeySet). 118 // 119 // It is safe to modify the contents of the arguments after DeleteRange 120 // returns. 121 DeleteRange(start, end []byte, o *WriteOptions) error 122 123 // LogData adds the specified to the batch. The data will be written to the 124 // WAL, but not added to memtables or sstables. Log data is never indexed, 125 // which makes it useful for testing WAL performance. 126 // 127 // It is safe to modify the contents of the argument after LogData returns. 128 LogData(data []byte, opts *WriteOptions) error 129 130 // Merge merges the value for the given key. The details of the merge are 131 // dependent upon the configured merge operation. 132 // 133 // It is safe to modify the contents of the arguments after Merge returns. 134 Merge(key, value []byte, o *WriteOptions) error 135 136 // Set sets the value for the given key. It overwrites any previous value 137 // for that key; a DB is not a multi-map. 138 // 139 // It is safe to modify the contents of the arguments after Set returns. 140 Set(key, value []byte, o *WriteOptions) error 141 142 // RangeKeySet sets a range key mapping the key range [start, end) at the MVCC 143 // timestamp suffix to value. The suffix is optional. If any portion of the key 144 // range [start, end) is already set by a range key with the same suffix value, 145 // RangeKeySet overrides it. 146 // 147 // It is safe to modify the contents of the arguments after RangeKeySet returns. 148 RangeKeySet(start, end, suffix, value []byte, opts *WriteOptions) error 149 150 // RangeKeyUnset removes a range key mapping the key range [start, end) at the 151 // MVCC timestamp suffix. The suffix may be omitted to remove an unsuffixed 152 // range key. RangeKeyUnset only removes portions of range keys that fall within 153 // the [start, end) key span, and only range keys with suffixes that exactly 154 // match the unset suffix. 155 // 156 // It is safe to modify the contents of the arguments after RangeKeyUnset 157 // returns. 158 RangeKeyUnset(start, end, suffix []byte, opts *WriteOptions) error 159 160 // RangeKeyDelete deletes all of the range keys in the range [start,end) 161 // (inclusive on start, exclusive on end). It does not delete point keys (for 162 // that use DeleteRange). RangeKeyDelete removes all range keys within the 163 // bounds, including those with or without suffixes. 164 // 165 // It is safe to modify the contents of the arguments after RangeKeyDelete 166 // returns. 167 RangeKeyDelete(start, end []byte, opts *WriteOptions) error 168 } 169 170 // CPUWorkPermissionGranter is used to request permission to opportunistically 171 // use additional CPUs to speed up internal background work. Each granted "proc" 172 // can be used to spin up a CPU bound goroutine, i.e, if scheduled each such 173 // goroutine can consume one P in the goroutine scheduler. The calls to 174 // ReturnProcs can be a bit delayed, since Pebble interacts with this interface 175 // in a coarse manner. So one should assume that the total number of granted 176 // procs is a non tight upper bound on the CPU that will get consumed. 177 type CPUWorkPermissionGranter interface { 178 TryGetProcs(count int) int 179 ReturnProcs(count int) 180 } 181 182 // DB provides a concurrent, persistent ordered key/value store. 183 // 184 // A DB's basic operations (Get, Set, Delete) should be self-explanatory. Get 185 // and Delete will return ErrNotFound if the requested key is not in the store. 186 // Callers are free to ignore this error. 187 // 188 // A DB also allows for iterating over the key/value pairs in key order. If d 189 // is a DB, the code below prints all key/value pairs whose keys are 'greater 190 // than or equal to' k: 191 // 192 // iter := d.NewIter(readOptions) 193 // for iter.SeekGE(k); iter.Valid(); iter.Next() { 194 // fmt.Printf("key=%q value=%q\n", iter.Key(), iter.Value()) 195 // } 196 // return iter.Close() 197 // 198 // The Options struct holds the optional parameters for the DB, including a 199 // Comparer to define a 'less than' relationship over keys. It is always valid 200 // to pass a nil *Options, which means to use the default parameter values. Any 201 // zero field of a non-nil *Options also means to use the default value for 202 // that parameter. Thus, the code below uses a custom Comparer, but the default 203 // values for every other parameter: 204 // 205 // db := bitalostable.Open(&Options{ 206 // Comparer: myComparer, 207 // }) 208 type DB struct { 209 // WARNING: The following struct `atomic` contains fields which are accessed 210 // atomically. 211 // 212 // Go allocations are guaranteed to be 64-bit aligned which we take advantage 213 // of by placing the 64-bit fields which we access atomically at the beginning 214 // of the DB struct. For more information, see https://golang.org/pkg/sync/atomic/#pkg-note-BUG. 215 atomic struct { 216 // The count and size of referenced memtables. This includes memtables 217 // present in DB.mu.mem.queue, as well as memtables that have been flushed 218 // but are still referenced by an inuse readState. 219 memTableCount int64 220 memTableReserved int64 // number of bytes reserved in the cache for memtables 221 222 // The size of the current log file (i.e. db.mu.log.queue[len(queue)-1]. 223 logSize uint64 224 225 // The number of bytes available on disk. 226 diskAvailBytes uint64 227 } 228 229 cacheID uint64 230 dirname string 231 walDirname string 232 opts *Options 233 cmp Compare 234 equal Equal 235 merge Merge 236 split Split 237 abbreviatedKey AbbreviatedKey 238 // The threshold for determining when a batch is "large" and will skip being 239 // inserted into a memtable. 240 largeBatchThreshold int 241 // The current OPTIONS file number. 242 optionsFileNum FileNum 243 // The on-disk size of the current OPTIONS file. 244 optionsFileSize uint64 245 246 fileLock io.Closer 247 dataDir vfs.File 248 walDir vfs.File 249 250 tableCache *tableCacheContainer 251 newIters tableNewIters 252 tableNewRangeKeyIter keyspan.TableNewSpanIter 253 254 commit *commitPipeline 255 256 // readState provides access to the state needed for reading without needing 257 // to acquire DB.mu. 258 readState struct { 259 sync.RWMutex 260 val *readState 261 } 262 // logRecycler holds a set of log file numbers that are available for 263 // reuse. Writing to a recycled log file is faster than to a new log file on 264 // some common filesystems (xfs, and ext3/4) due to avoiding metadata 265 // updates. 266 logRecycler logRecycler 267 268 closed *atomic.Value 269 closedCh chan struct{} 270 271 deletionLimiter limiter 272 273 // Async deletion jobs spawned by cleaners increment this WaitGroup, and 274 // call Done when completed. Once `d.mu.cleaning` is false, the db.Close() 275 // goroutine needs to call Wait on this WaitGroup to ensure all cleaning 276 // and deleting goroutines have finished running. As deletion goroutines 277 // could grab db.mu, it must *not* be held while deleters.Wait() is called. 278 deleters sync.WaitGroup 279 280 // During an iterator close, we may asynchronously schedule read compactions. 281 // We want to wait for those goroutines to finish, before closing the DB. 282 // compactionShedulers.Wait() should not be called while the DB.mu is held. 283 compactionSchedulers sync.WaitGroup 284 285 // The main mutex protecting internal DB state. This mutex encompasses many 286 // fields because those fields need to be accessed and updated atomically. In 287 // particular, the current version, log.*, mem.*, and snapshot list need to 288 // be accessed and updated atomically during compaction. 289 // 290 // Care is taken to avoid holding DB.mu during IO operations. Accomplishing 291 // this sometimes requires releasing DB.mu in a method that was called with 292 // it held. See versionSet.logAndApply() and DB.makeRoomForWrite() for 293 // examples. This is a common pattern, so be careful about expectations that 294 // DB.mu will be held continuously across a set of calls. 295 mu struct { 296 sync.Mutex 297 298 formatVers struct { 299 // vers is the database's current format major version. 300 // Backwards-incompatible features are gated behind new 301 // format major versions and not enabled until a database's 302 // version is ratcheted upwards. 303 vers FormatMajorVersion 304 // marker is the atomic marker for the format major version. 305 // When a database's version is ratcheted upwards, the 306 // marker is moved in order to atomically record the new 307 // version. 308 marker *atomicfs.Marker 309 // ratcheting when set to true indicates that the database is 310 // currently in the process of ratcheting the format major version 311 // to vers + 1. As a part of ratcheting the format major version, 312 // migrations may drop and re-acquire the mutex. 313 ratcheting bool 314 } 315 316 // The ID of the next job. Job IDs are passed to event listener 317 // notifications and act as a mechanism for tying together the events and 318 // log messages for a single job such as a flush, compaction, or file 319 // ingestion. Job IDs are not serialized to disk or used for correctness. 320 nextJobID int 321 322 // The collection of immutable versions and state about the log and visible 323 // sequence numbers. Use the pointer here to ensure the atomic fields in 324 // version set are aligned properly. 325 versions *versionSet 326 327 log struct { 328 // The queue of logs, containing both flushed and unflushed logs. The 329 // flushed logs will be a prefix, the unflushed logs a suffix. The 330 // delimeter between flushed and unflushed logs is 331 // versionSet.minUnflushedLogNum. 332 queue []fileInfo 333 // The number of input bytes to the log. This is the raw size of the 334 // batches written to the WAL, without the overhead of the record 335 // envelopes. 336 bytesIn uint64 337 // The LogWriter is protected by commitPipeline.mu. This allows log 338 // writes to be performed without holding DB.mu, but requires both 339 // commitPipeline.mu and DB.mu to be held when rotating the WAL/memtable 340 // (i.e. makeRoomForWrite). 341 *record.LogWriter 342 // Can be nil. 343 metrics *record.LogWriterMetrics 344 } 345 346 mem struct { 347 // Condition variable used to serialize memtable switching. See 348 // DB.makeRoomForWrite(). 349 cond sync.Cond 350 // The current mutable memTable. 351 mutable *memTable 352 // Queue of flushables (the mutable memtable is at end). Elements are 353 // added to the end of the slice and removed from the beginning. Once an 354 // index is set it is never modified making a fixed slice immutable and 355 // safe for concurrent reads. 356 queue flushableList 357 // True when the memtable is actively being switched. Both mem.mutable and 358 // log.LogWriter are invalid while switching is true. 359 switching bool 360 // nextSize is the size of the next memtable. The memtable size starts at 361 // min(256KB,Options.MemTableSize) and doubles each time a new memtable 362 // is allocated up to Options.MemTableSize. This reduces the memory 363 // footprint of memtables when lots of DB instances are used concurrently 364 // in test environments. 365 nextSize int 366 } 367 368 compact struct { 369 // Condition variable used to signal when a flush or compaction has 370 // completed. Used by the write-stall mechanism to wait for the stall 371 // condition to clear. See DB.makeRoomForWrite(). 372 cond sync.Cond 373 // True when a flush is in progress. 374 flushing bool 375 // The number of ongoing compactions. 376 compactingCount int 377 // The list of deletion hints, suggesting ranges for delete-only 378 // compactions. 379 deletionHints []deleteCompactionHint 380 // The list of manual compactions. The next manual compaction to perform 381 // is at the start of the list. New entries are added to the end. 382 manual []*manualCompaction 383 // inProgress is the set of in-progress flushes and compactions. 384 inProgress map[*compaction]struct{} 385 386 // rescheduleReadCompaction indicates to an iterator that a read compaction 387 // should be scheduled. 388 rescheduleReadCompaction bool 389 390 // readCompactions is a readCompactionQueue which keeps track of the 391 // compactions which we might have to perform. 392 readCompactions readCompactionQueue 393 394 // Flush throughput metric. 395 flushWriteThroughput ThroughputMetric 396 // The idle start time for the flush "loop", i.e., when the flushing 397 // bool above transitions to false. 398 noOngoingFlushStartTime time.Time 399 } 400 401 cleaner struct { 402 // Condition variable used to signal the completion of a file cleaning 403 // operation or an increment to the value of disabled. File cleaning operations are 404 // serialized, and a caller trying to do a file cleaning operation may wait 405 // until the ongoing one is complete. 406 cond sync.Cond 407 // True when a file cleaning operation is in progress. False does not necessarily 408 // mean all cleaning jobs have completed; see the comment on d.deleters. 409 cleaning bool 410 // Non-zero when file cleaning is disabled. The disabled count acts as a 411 // reference count to prohibit file cleaning. See 412 // DB.{disable,Enable}FileDeletions(). 413 disabled int 414 } 415 416 // The list of active snapshots. 417 snapshots snapshotList 418 419 tableStats struct { 420 // Condition variable used to signal the completion of a 421 // job to collect table stats. 422 cond sync.Cond 423 // True when a stat collection operation is in progress. 424 loading bool 425 // True if stat collection has loaded statistics for all tables 426 // other than those listed explcitly in pending. This flag starts 427 // as false when a database is opened and flips to true once stat 428 // collection has caught up. 429 loadedInitial bool 430 // A slice of files for which stats have not been computed. 431 // Compactions, ingests, flushes append files to be processed. An 432 // active stat collection goroutine clears the list and processes 433 // them. 434 pending []manifest.NewFileEntry 435 } 436 437 tableValidation struct { 438 // cond is a condition variable used to signal the completion of a 439 // job to validate one or more sstables. 440 cond sync.Cond 441 // pending is a slice of metadata for sstables waiting to be 442 // validated. 443 pending []newFileEntry 444 // validating is set to true when validation is running. 445 validating bool 446 } 447 } 448 449 // Normally equal to time.Now() but may be overridden in tests. 450 timeNow func() time.Time 451 } 452 453 var _ Reader = (*DB)(nil) 454 var _ Writer = (*DB)(nil) 455 456 // Get gets the value for the given key. It returns ErrNotFound if the DB does 457 // not contain the key. 458 // 459 // The caller should not modify the contents of the returned slice, but it is 460 // safe to modify the contents of the argument after Get returns. The returned 461 // slice will remain valid until the returned Closer is closed. On success, the 462 // caller MUST call closer.Close() or a memory leak will occur. 463 func (d *DB) Get(key []byte) ([]byte, io.Closer, error) { 464 return d.getInternal(key, nil /* batch */, nil /* snapshot */) 465 } 466 467 type getIterAlloc struct { 468 dbi Iterator 469 keyBuf []byte 470 get getIter 471 } 472 473 var getIterAllocPool = sync.Pool{ 474 New: func() interface{} { 475 return &getIterAlloc{} 476 }, 477 } 478 479 func (d *DB) getInternal(key []byte, b *Batch, s *Snapshot) ([]byte, io.Closer, error) { 480 if err := d.closed.Load(); err != nil { 481 panic(err) 482 } 483 484 // Grab and reference the current readState. This prevents the underlying 485 // files in the associated version from being deleted if there is a current 486 // compaction. The readState is unref'd by Iterator.Close(). 487 readState := d.loadReadState() 488 489 // Determine the seqnum to read at after grabbing the read state (current and 490 // memtables) above. 491 var seqNum uint64 492 if s != nil { 493 seqNum = s.seqNum 494 } else { 495 seqNum = atomic.LoadUint64(&d.mu.versions.atomic.visibleSeqNum) 496 } 497 498 buf := getIterAllocPool.Get().(*getIterAlloc) 499 500 get := &buf.get 501 *get = getIter{ 502 logger: d.opts.Logger, 503 cmp: d.cmp, 504 equal: d.equal, 505 newIters: d.newIters, 506 snapshot: seqNum, 507 key: key, 508 batch: b, 509 mem: readState.memtables, 510 l0: readState.current.L0SublevelFiles, 511 version: readState.current, 512 } 513 514 // Strip off memtables which cannot possibly contain the seqNum being read 515 // at. 516 for len(get.mem) > 0 { 517 n := len(get.mem) 518 if logSeqNum := get.mem[n-1].logSeqNum; logSeqNum < seqNum { 519 break 520 } 521 get.mem = get.mem[:n-1] 522 } 523 524 i := &buf.dbi 525 pointIter := get 526 *i = Iterator{ 527 getIterAlloc: buf, 528 iter: pointIter, 529 pointIter: pointIter, 530 merge: d.merge, 531 comparer: *d.opts.Comparer, 532 readState: readState, 533 keyBuf: buf.keyBuf, 534 } 535 536 if !i.First() { 537 err := i.Close() 538 if err != nil { 539 return nil, nil, err 540 } 541 return nil, nil, ErrNotFound 542 } 543 return i.Value(), i, nil 544 } 545 546 // Set sets the value for the given key. It overwrites any previous value 547 // for that key; a DB is not a multi-map. 548 // 549 // It is safe to modify the contents of the arguments after Set returns. 550 func (d *DB) Set(key, value []byte, opts *WriteOptions) error { 551 b := newBatch(d) 552 _ = b.Set(key, value, opts) 553 if err := d.Apply(b, opts); err != nil { 554 return err 555 } 556 // Only release the batch on success. 557 b.release() 558 return nil 559 } 560 561 // Delete deletes the value for the given key. Deletes are blind all will 562 // succeed even if the given key does not exist. 563 // 564 // It is safe to modify the contents of the arguments after Delete returns. 565 func (d *DB) Delete(key []byte, opts *WriteOptions) error { 566 b := newBatch(d) 567 _ = b.Delete(key, opts) 568 if err := d.Apply(b, opts); err != nil { 569 return err 570 } 571 // Only release the batch on success. 572 b.release() 573 return nil 574 } 575 576 // SingleDelete adds an action to the batch that single deletes the entry for key. 577 // See Writer.SingleDelete for more details on the semantics of SingleDelete. 578 // 579 // It is safe to modify the contents of the arguments after SingleDelete returns. 580 func (d *DB) SingleDelete(key []byte, opts *WriteOptions) error { 581 b := newBatch(d) 582 _ = b.SingleDelete(key, opts) 583 if err := d.Apply(b, opts); err != nil { 584 return err 585 } 586 // Only release the batch on success. 587 b.release() 588 return nil 589 } 590 591 // DeleteRange deletes all of the keys (and values) in the range [start,end) 592 // (inclusive on start, exclusive on end). 593 // 594 // It is safe to modify the contents of the arguments after DeleteRange 595 // returns. 596 func (d *DB) DeleteRange(start, end []byte, opts *WriteOptions) error { 597 b := newBatch(d) 598 _ = b.DeleteRange(start, end, opts) 599 if err := d.Apply(b, opts); err != nil { 600 return err 601 } 602 // Only release the batch on success. 603 b.release() 604 return nil 605 } 606 607 // Merge adds an action to the DB that merges the value at key with the new 608 // value. The details of the merge are dependent upon the configured merge 609 // operator. 610 // 611 // It is safe to modify the contents of the arguments after Merge returns. 612 func (d *DB) Merge(key, value []byte, opts *WriteOptions) error { 613 b := newBatch(d) 614 _ = b.Merge(key, value, opts) 615 if err := d.Apply(b, opts); err != nil { 616 return err 617 } 618 // Only release the batch on success. 619 b.release() 620 return nil 621 } 622 623 // LogData adds the specified to the batch. The data will be written to the 624 // WAL, but not added to memtables or sstables. Log data is never indexed, 625 // which makes it useful for testing WAL performance. 626 // 627 // It is safe to modify the contents of the argument after LogData returns. 628 func (d *DB) LogData(data []byte, opts *WriteOptions) error { 629 b := newBatch(d) 630 _ = b.LogData(data, opts) 631 if err := d.Apply(b, opts); err != nil { 632 return err 633 } 634 // Only release the batch on success. 635 b.release() 636 return nil 637 } 638 639 // RangeKeySet sets a range key mapping the key range [start, end) at the MVCC 640 // timestamp suffix to value. The suffix is optional. If any portion of the key 641 // range [start, end) is already set by a range key with the same suffix value, 642 // RangeKeySet overrides it. 643 // 644 // It is safe to modify the contents of the arguments after RangeKeySet returns. 645 func (d *DB) RangeKeySet(start, end, suffix, value []byte, opts *WriteOptions) error { 646 b := newBatch(d) 647 _ = b.RangeKeySet(start, end, suffix, value, opts) 648 if err := d.Apply(b, opts); err != nil { 649 return err 650 } 651 // Only release the batch on success. 652 b.release() 653 return nil 654 } 655 656 // RangeKeyUnset removes a range key mapping the key range [start, end) at the 657 // MVCC timestamp suffix. The suffix may be omitted to remove an unsuffixed 658 // range key. RangeKeyUnset only removes portions of range keys that fall within 659 // the [start, end) key span, and only range keys with suffixes that exactly 660 // match the unset suffix. 661 // 662 // It is safe to modify the contents of the arguments after RangeKeyUnset 663 // returns. 664 func (d *DB) RangeKeyUnset(start, end, suffix []byte, opts *WriteOptions) error { 665 b := newBatch(d) 666 _ = b.RangeKeyUnset(start, end, suffix, opts) 667 if err := d.Apply(b, opts); err != nil { 668 return err 669 } 670 // Only release the batch on success. 671 b.release() 672 return nil 673 } 674 675 // RangeKeyDelete deletes all of the range keys in the range [start,end) 676 // (inclusive on start, exclusive on end). It does not delete point keys (for 677 // that use DeleteRange). RangeKeyDelete removes all range keys within the 678 // bounds, including those with or without suffixes. 679 // 680 // It is safe to modify the contents of the arguments after RangeKeyDelete 681 // returns. 682 func (d *DB) RangeKeyDelete(start, end []byte, opts *WriteOptions) error { 683 b := newBatch(d) 684 _ = b.RangeKeyDelete(start, end, opts) 685 if err := d.Apply(b, opts); err != nil { 686 return err 687 } 688 // Only release the batch on success. 689 b.release() 690 return nil 691 } 692 693 // Apply the operations contained in the batch to the DB. If the batch is large 694 // the contents of the batch may be retained by the database. If that occurs 695 // the batch contents will be cleared preventing the caller from attempting to 696 // reuse them. 697 // 698 // It is safe to modify the contents of the arguments after Apply returns. 699 func (d *DB) Apply(batch *Batch, opts *WriteOptions) error { 700 if err := d.closed.Load(); err != nil { 701 panic(err) 702 } 703 if atomic.LoadUint32(&batch.applied) != 0 { 704 panic("bitalostable: batch already applied") 705 } 706 if d.opts.ReadOnly { 707 return ErrReadOnly 708 } 709 if batch.db != nil && batch.db != d { 710 panic(fmt.Sprintf("bitalostable: batch db mismatch: %p != %p", batch.db, d)) 711 } 712 713 sync := opts.GetSync() 714 if sync && d.opts.DisableWAL { 715 return errors.New("bitalostable: WAL disabled") 716 } 717 718 if batch.countRangeKeys > 0 { 719 if d.split == nil { 720 return errNoSplit 721 } 722 if d.FormatMajorVersion() < FormatRangeKeys { 723 panic(fmt.Sprintf( 724 "bitalostable: range keys require at least format major version %d (current: %d)", 725 FormatRangeKeys, d.FormatMajorVersion(), 726 )) 727 } 728 729 // TODO(jackson): Assert that all range key operands are suffixless. 730 } 731 732 if batch.db == nil { 733 batch.refreshMemTableSize() 734 } 735 if int(batch.memTableSize) >= d.largeBatchThreshold || batch.isFlush { 736 batch.flushable = newFlushableBatch(batch, d.opts.Comparer) 737 } 738 if err := d.commit.Commit(batch, sync); err != nil { 739 // There isn't much we can do on an error here. The commit pipeline will be 740 // horked at this point. 741 d.opts.Logger.Fatalf("%v", err) 742 } 743 // If this is a large batch, we need to clear the batch contents as the 744 // flushable batch may still be present in the flushables queue. 745 // 746 // TODO(peter): Currently large batches are written to the WAL. We could 747 // skip the WAL write and instead wait for the large batch to be flushed to 748 // an sstable. For a 100 MB batch, this might actually be faster. For a 1 749 // GB batch this is almost certainly faster. 750 if batch.flushable != nil { 751 batch.data = nil 752 batch.alloc = nil 753 } 754 return nil 755 } 756 757 func (d *DB) commitApply(b *Batch, mem *memTable) error { 758 if b.flushable != nil { 759 // This is a large batch which was already added to the immutable queue. 760 return nil 761 } 762 err := mem.apply(b, b.SeqNum()) 763 if err != nil { 764 return err 765 } 766 767 // If the batch contains range tombstones and the database is configured 768 // to flush range deletions, schedule a delayed flush so that disk space 769 // may be reclaimed without additional writes or an explicit flush. 770 if b.countRangeDels > 0 && d.opts.FlushDelayDeleteRange > 0 { 771 d.mu.Lock() 772 d.maybeScheduleDelayedFlush(mem, d.opts.FlushDelayDeleteRange) 773 d.mu.Unlock() 774 } 775 776 // If the batch contains range keys and the database is configured to flush 777 // range keys, schedule a delayed flush so that the range keys are cleared 778 // from the memtable. 779 if b.countRangeKeys > 0 && d.opts.FlushDelayRangeKey > 0 { 780 d.mu.Lock() 781 d.maybeScheduleDelayedFlush(mem, d.opts.FlushDelayRangeKey) 782 d.mu.Unlock() 783 } 784 785 if mem.writerUnref() { 786 d.mu.Lock() 787 d.maybeScheduleFlush(true) 788 d.mu.Unlock() 789 } 790 return nil 791 } 792 793 func (d *DB) commitWrite(b *Batch, syncWG *sync.WaitGroup, syncErr *error) (*memTable, error) { 794 var size int64 795 repr := b.Repr() 796 797 if b.flushable != nil { 798 // We have a large batch. Such batches are special in that they don't get 799 // added to the memtable, and are instead inserted into the queue of 800 // memtables. The call to makeRoomForWrite with this batch will force the 801 // current memtable to be flushed. We want the large batch to be part of 802 // the same log, so we add it to the WAL here, rather than after the call 803 // to makeRoomForWrite(). 804 // 805 // Set the sequence number since it was not set to the correct value earlier 806 // (see comment in newFlushableBatch()). 807 b.flushable.setSeqNum(b.SeqNum()) 808 if !d.opts.DisableWAL && !b.isFlush { 809 var err error 810 size, err = d.mu.log.SyncRecord(repr, syncWG, syncErr) 811 if err != nil { 812 panic(err) 813 } 814 } 815 } 816 817 d.mu.Lock() 818 819 // Switch out the memtable if there was not enough room to store the batch. 820 err := d.makeRoomForWrite(b, true) 821 822 if err == nil && !d.opts.DisableWAL { 823 d.mu.log.bytesIn += uint64(len(repr)) 824 } 825 826 // Grab a reference to the memtable while holding DB.mu. Note that for 827 // non-flushable batches (b.flushable == nil) makeRoomForWrite() added a 828 // reference to the memtable which will prevent it from being flushed until 829 // we unreference it. This reference is dropped in DB.commitApply(). 830 mem := d.mu.mem.mutable 831 832 d.mu.Unlock() 833 if err != nil { 834 return nil, err 835 } 836 837 if d.opts.DisableWAL { 838 return mem, nil 839 } 840 841 if b.flushable == nil { 842 size, err = d.mu.log.SyncRecord(repr, syncWG, syncErr) 843 if err != nil { 844 panic(err) 845 } 846 } 847 848 atomic.StoreUint64(&d.atomic.logSize, uint64(size)) 849 return mem, err 850 } 851 852 type iterAlloc struct { 853 dbi Iterator 854 keyBuf []byte 855 boundsBuf [2][]byte 856 prefixOrFullSeekKey []byte 857 merging mergingIter 858 mlevels [3 + numLevels]mergingIterLevel 859 levels [3 + numLevels]levelIter 860 } 861 862 var iterAllocPool = sync.Pool{ 863 New: func() interface{} { 864 return &iterAlloc{} 865 }, 866 } 867 868 // newIterInternal constructs a new iterator, merging in batch iterators as an extra 869 // level. 870 func (d *DB) newIterInternal(batch *Batch, s *Snapshot, o *IterOptions) *Iterator { 871 if err := d.closed.Load(); err != nil { 872 panic(err) 873 } 874 if o.rangeKeys() { 875 if d.FormatMajorVersion() < FormatRangeKeys { 876 panic(fmt.Sprintf( 877 "bitalostable: range keys require at least format major version %d (current: %d)", 878 FormatRangeKeys, d.FormatMajorVersion(), 879 )) 880 } 881 } 882 if o != nil && o.RangeKeyMasking.Suffix != nil && o.KeyTypes != IterKeyTypePointsAndRanges { 883 panic("bitalostable: range key masking requires IterKeyTypePointsAndRanges") 884 } 885 if (batch != nil || s != nil) && (o != nil && o.OnlyReadGuaranteedDurable) { 886 // We could add support for OnlyReadGuaranteedDurable on snapshots if 887 // there was a need: this would require checking that the sequence number 888 // of the snapshot has been flushed, by comparing with 889 // DB.mem.queue[0].logSeqNum. 890 panic("OnlyReadGuaranteedDurable is not supported for batches or snapshots") 891 } 892 // Grab and reference the current readState. This prevents the underlying 893 // files in the associated version from being deleted if there is a current 894 // compaction. The readState is unref'd by Iterator.Close(). 895 readState := d.loadReadState() 896 897 // Determine the seqnum to read at after grabbing the read state (current and 898 // memtables) above. 899 var seqNum uint64 900 if s != nil { 901 seqNum = s.seqNum 902 } else { 903 seqNum = atomic.LoadUint64(&d.mu.versions.atomic.visibleSeqNum) 904 } 905 906 // Bundle various structures under a single umbrella in order to allocate 907 // them together. 908 buf := iterAllocPool.Get().(*iterAlloc) 909 dbi := &buf.dbi 910 *dbi = Iterator{ 911 alloc: buf, 912 merge: d.merge, 913 comparer: *d.opts.Comparer, 914 readState: readState, 915 keyBuf: buf.keyBuf, 916 prefixOrFullSeekKey: buf.prefixOrFullSeekKey, 917 boundsBuf: buf.boundsBuf, 918 batch: batch, 919 newIters: d.newIters, 920 newIterRangeKey: d.tableNewRangeKeyIter, 921 seqNum: seqNum, 922 } 923 if o != nil { 924 dbi.opts = *o 925 dbi.saveBounds(o.LowerBound, o.UpperBound) 926 } 927 dbi.opts.logger = d.opts.Logger 928 if batch != nil { 929 dbi.batchSeqNum = dbi.batch.nextSeqNum() 930 } 931 return finishInitializingIter(buf) 932 } 933 934 // finishInitializingIter is a helper for doing the non-trivial initialization 935 // of an Iterator. It's invoked to perform the initial initialization of an 936 // Iterator during NewIter or Clone, and to perform reinitialization due to a 937 // change in IterOptions by a call to Iterator.SetOptions. 938 func finishInitializingIter(buf *iterAlloc) *Iterator { 939 // Short-hand. 940 dbi := &buf.dbi 941 memtables := dbi.readState.memtables 942 if dbi.opts.OnlyReadGuaranteedDurable { 943 memtables = nil 944 } else { 945 // We only need to read from memtables which contain sequence numbers older 946 // than seqNum. Trim off newer memtables. 947 for i := len(memtables) - 1; i >= 0; i-- { 948 if logSeqNum := memtables[i].logSeqNum; logSeqNum < dbi.seqNum { 949 break 950 } 951 memtables = memtables[:i] 952 } 953 } 954 955 if dbi.opts.pointKeys() { 956 // Construct the point iterator, initializing dbi.pointIter to point to 957 // dbi.merging. If this is called during a SetOptions call and this 958 // Iterator has already initialized dbi.merging, constructPointIter is a 959 // noop and an initialized pointIter already exists in dbi.pointIter. 960 dbi.constructPointIter(memtables, buf) 961 dbi.iter = dbi.pointIter 962 } else { 963 dbi.iter = emptyIter 964 } 965 966 if dbi.opts.rangeKeys() { 967 dbi.rangeKeyMasking.init(dbi, dbi.comparer.Compare, dbi.comparer.Split) 968 969 // When iterating over both point and range keys, don't create the 970 // range-key iterator stack immediately if we can avoid it. This 971 // optimization takes advantage of the expected sparseness of range 972 // keys, and configures the point-key iterator to dynamically switch to 973 // combined iteration when it observes a file containing range keys. 974 // 975 // Lazy combined iteration is not possible if a batch or a memtable 976 // contains any range keys. 977 useLazyCombinedIteration := dbi.rangeKey == nil && 978 dbi.opts.KeyTypes == IterKeyTypePointsAndRanges && 979 (dbi.batch == nil || dbi.batch.countRangeKeys == 0) 980 if useLazyCombinedIteration { 981 // The user requested combined iteration, and there's no indexed 982 // batch currently containing range keys that would prevent lazy 983 // combined iteration. Check the memtables to see if they contain 984 // any range keys. 985 for i := range memtables { 986 if memtables[i].containsRangeKeys() { 987 useLazyCombinedIteration = false 988 break 989 } 990 } 991 } 992 993 if useLazyCombinedIteration { 994 dbi.lazyCombinedIter = lazyCombinedIter{ 995 parent: dbi, 996 pointIter: dbi.pointIter, 997 combinedIterState: combinedIterState{ 998 initialized: false, 999 }, 1000 } 1001 dbi.iter = &dbi.lazyCombinedIter 1002 } else { 1003 dbi.lazyCombinedIter.combinedIterState = combinedIterState{ 1004 initialized: true, 1005 } 1006 if dbi.rangeKey == nil { 1007 dbi.rangeKey = iterRangeKeyStateAllocPool.Get().(*iteratorRangeKeyState) 1008 dbi.rangeKey.init(dbi.comparer.Compare, dbi.comparer.Split, &dbi.opts) 1009 dbi.constructRangeKeyIter() 1010 } else { 1011 dbi.rangeKey.iterConfig.SetBounds(dbi.opts.LowerBound, dbi.opts.UpperBound) 1012 } 1013 1014 // Wrap the point iterator (currently dbi.iter) with an interleaving 1015 // iterator that interleaves range keys pulled from 1016 // dbi.rangeKey.rangeKeyIter. 1017 // 1018 // NB: The interleaving iterator is always reinitialized, even if 1019 // dbi already had an initialized range key iterator, in case the point 1020 // iterator changed or the range key masking suffix changed. 1021 dbi.rangeKey.iiter.Init(&dbi.comparer, dbi.iter, dbi.rangeKey.rangeKeyIter, 1022 &dbi.rangeKeyMasking, dbi.opts.LowerBound, dbi.opts.UpperBound) 1023 dbi.iter = &dbi.rangeKey.iiter 1024 } 1025 } else { 1026 // !dbi.opts.rangeKeys() 1027 // 1028 // Reset the combined iterator state. The initialized=true ensures the 1029 // iterator doesn't unnecessarily try to switch to combined iteration. 1030 dbi.lazyCombinedIter.combinedIterState = combinedIterState{initialized: true} 1031 } 1032 return dbi 1033 } 1034 1035 func (i *Iterator) constructPointIter(memtables flushableList, buf *iterAlloc) { 1036 if i.pointIter != nil { 1037 // Already have one. 1038 return 1039 } 1040 internalOpts := internalIterOpts{stats: &i.stats.InternalStats} 1041 if i.opts.RangeKeyMasking.Filter != nil { 1042 internalOpts.boundLimitedFilter = &i.rangeKeyMasking 1043 } 1044 1045 // Merging levels and levels from iterAlloc. 1046 mlevels := buf.mlevels[:0] 1047 levels := buf.levels[:0] 1048 1049 // We compute the number of levels needed ahead of time and reallocate a slice if 1050 // the array from the iterAlloc isn't large enough. Doing this allocation once 1051 // should improve the performance. 1052 numMergingLevels := 0 1053 numLevelIters := 0 1054 if i.batch != nil { 1055 numMergingLevels++ 1056 } 1057 numMergingLevels += len(memtables) 1058 1059 current := i.readState.current 1060 numMergingLevels += len(current.L0SublevelFiles) 1061 numLevelIters += len(current.L0SublevelFiles) 1062 for level := 1; level < len(current.Levels); level++ { 1063 if current.Levels[level].Empty() { 1064 continue 1065 } 1066 numMergingLevels++ 1067 numLevelIters++ 1068 } 1069 1070 if numMergingLevels > cap(mlevels) { 1071 mlevels = make([]mergingIterLevel, 0, numMergingLevels) 1072 } 1073 if numLevelIters > cap(levels) { 1074 levels = make([]levelIter, 0, numLevelIters) 1075 } 1076 1077 // Top-level is the batch, if any. 1078 if i.batch != nil { 1079 if i.batch.index == nil { 1080 // This isn't an indexed batch. Include an error iterator so that 1081 // the resulting iterator correctly surfaces ErrIndexed. 1082 mlevels = append(mlevels, mergingIterLevel{ 1083 iter: newErrorIter(ErrNotIndexed), 1084 rangeDelIter: newErrorKeyspanIter(ErrNotIndexed), 1085 }) 1086 } else { 1087 i.batch.initInternalIter(&i.opts, &i.batchPointIter, i.batchSeqNum) 1088 i.batch.initRangeDelIter(&i.opts, &i.batchRangeDelIter, i.batchSeqNum) 1089 // Only include the batch's rangedel iterator if it's non-empty. 1090 // This requires some subtle logic in the case a rangedel is later 1091 // written to the batch and the view of the batch is refreshed 1092 // during a call to SetOptions—in this case, we need to reconstruct 1093 // the point iterator to add the batch rangedel iterator. 1094 var rangeDelIter keyspan.FragmentIterator 1095 if i.batchRangeDelIter.Count() > 0 { 1096 rangeDelIter = &i.batchRangeDelIter 1097 } 1098 mlevels = append(mlevels, mergingIterLevel{ 1099 iter: &i.batchPointIter, 1100 rangeDelIter: rangeDelIter, 1101 }) 1102 } 1103 } 1104 1105 // Next are the memtables. 1106 for j := len(memtables) - 1; j >= 0; j-- { 1107 mem := memtables[j] 1108 mlevels = append(mlevels, mergingIterLevel{ 1109 iter: mem.newIter(&i.opts), 1110 rangeDelIter: mem.newRangeDelIter(&i.opts), 1111 }) 1112 } 1113 1114 // Next are the file levels: L0 sub-levels followed by lower levels. 1115 mlevelsIndex := len(mlevels) 1116 levelsIndex := len(levels) 1117 mlevels = mlevels[:numMergingLevels] 1118 levels = levels[:numLevelIters] 1119 addLevelIterForFiles := func(files manifest.LevelIterator, level manifest.Level) { 1120 li := &levels[levelsIndex] 1121 1122 li.init(i.opts, i.comparer.Compare, i.comparer.Split, i.newIters, files, level, internalOpts) 1123 li.initRangeDel(&mlevels[mlevelsIndex].rangeDelIter) 1124 li.initBoundaryContext(&mlevels[mlevelsIndex].levelIterBoundaryContext) 1125 li.initCombinedIterState(&i.lazyCombinedIter.combinedIterState) 1126 mlevels[mlevelsIndex].iter = li 1127 1128 levelsIndex++ 1129 mlevelsIndex++ 1130 } 1131 1132 // Add level iterators for the L0 sublevels, iterating from newest to 1133 // oldest. 1134 for i := len(current.L0SublevelFiles) - 1; i >= 0; i-- { 1135 addLevelIterForFiles(current.L0SublevelFiles[i].Iter(), manifest.L0Sublevel(i)) 1136 } 1137 1138 // Add level iterators for the non-empty non-L0 levels. 1139 for level := 1; level < len(current.Levels); level++ { 1140 if current.Levels[level].Empty() { 1141 continue 1142 } 1143 addLevelIterForFiles(current.Levels[level].Iter(), manifest.Level(level)) 1144 } 1145 buf.merging.init(&i.opts, &i.stats.InternalStats, i.comparer.Compare, i.comparer.Split, mlevels...) 1146 buf.merging.snapshot = i.seqNum 1147 buf.merging.elideRangeTombstones = true 1148 buf.merging.combinedIterState = &i.lazyCombinedIter.combinedIterState 1149 i.pointIter = &buf.merging 1150 } 1151 1152 // NewBatch returns a new empty write-only batch. Any reads on the batch will 1153 // return an error. If the batch is committed it will be applied to the DB. 1154 func (d *DB) NewBatch() *Batch { 1155 return newBatch(d) 1156 } 1157 1158 func (d *DB) NewFlushBatch(n int) *Batch { 1159 return newFlushBatch(d, n) 1160 } 1161 1162 // NewIndexedBatch returns a new empty read-write batch. Any reads on the batch 1163 // will read from both the batch and the DB. If the batch is committed it will 1164 // be applied to the DB. An indexed batch is slower that a non-indexed batch 1165 // for insert operations. If you do not need to perform reads on the batch, use 1166 // NewBatch instead. 1167 func (d *DB) NewIndexedBatch() *Batch { 1168 return newIndexedBatch(d, d.opts.Comparer) 1169 } 1170 1171 // NewIter returns an iterator that is unpositioned (Iterator.Valid() will 1172 // return false). The iterator can be positioned via a call to SeekGE, SeekLT, 1173 // First or Last. The iterator provides a point-in-time view of the current DB 1174 // state. This view is maintained by preventing file deletions and preventing 1175 // memtables referenced by the iterator from being deleted. Using an iterator 1176 // to maintain a long-lived point-in-time view of the DB state can lead to an 1177 // apparent memory and disk usage leak. Use snapshots (see NewSnapshot) for 1178 // point-in-time snapshots which avoids these problems. 1179 func (d *DB) NewIter(o *IterOptions) *Iterator { 1180 return d.newIterInternal(nil /* batch */, nil /* snapshot */, o) 1181 } 1182 1183 // NewSnapshot returns a point-in-time view of the current DB state. Iterators 1184 // created with this handle will all observe a stable snapshot of the current 1185 // DB state. The caller must call Snapshot.Close() when the snapshot is no 1186 // longer needed. Snapshots are not persisted across DB restarts (close -> 1187 // open). Unlike the implicit snapshot maintained by an iterator, a snapshot 1188 // will not prevent memtables from being released or sstables from being 1189 // deleted. Instead, a snapshot prevents deletion of sequence numbers 1190 // referenced by the snapshot. 1191 func (d *DB) NewSnapshot() *Snapshot { 1192 if err := d.closed.Load(); err != nil { 1193 panic(err) 1194 } 1195 1196 d.mu.Lock() 1197 s := &Snapshot{ 1198 db: d, 1199 seqNum: atomic.LoadUint64(&d.mu.versions.atomic.visibleSeqNum), 1200 } 1201 d.mu.snapshots.pushBack(s) 1202 d.mu.Unlock() 1203 return s 1204 } 1205 1206 // Close closes the DB. 1207 // 1208 // It is not safe to close a DB until all outstanding iterators are closed 1209 // or to call Close concurrently with any other DB method. It is not valid 1210 // to call any of a DB's methods after the DB has been closed. 1211 func (d *DB) Close() error { 1212 // Lock the commit pipeline for the duration of Close. This prevents a race 1213 // with makeRoomForWrite. Rotating the WAL in makeRoomForWrite requires 1214 // dropping d.mu several times for I/O. If Close only holds d.mu, an 1215 // in-progress WAL rotation may re-acquire d.mu only once the database is 1216 // closed. 1217 // 1218 // Additionally, locking the commit pipeline makes it more likely that 1219 // (illegal) concurrent writes will observe d.closed.Load() != nil, creating 1220 // more understable panics if the database is improperly used concurrently 1221 // during Close. 1222 d.commit.mu.Lock() 1223 defer d.commit.mu.Unlock() 1224 d.mu.Lock() 1225 defer d.mu.Unlock() 1226 if err := d.closed.Load(); err != nil { 1227 panic(err) 1228 } 1229 1230 // Clear the finalizer that is used to check that an unreferenced DB has been 1231 // closed. We're closing the DB here, so the check performed by that 1232 // finalizer isn't necessary. 1233 // 1234 // Note: this is a no-op if invariants are disabled or race is enabled. 1235 invariants.SetFinalizer(d.closed, nil) 1236 1237 d.closed.Store(errors.WithStack(ErrClosed)) 1238 close(d.closedCh) 1239 1240 defer d.opts.Cache.Unref() 1241 1242 for d.mu.compact.compactingCount > 0 || d.mu.compact.flushing { 1243 d.mu.compact.cond.Wait() 1244 } 1245 for d.mu.tableStats.loading { 1246 d.mu.tableStats.cond.Wait() 1247 } 1248 for d.mu.tableValidation.validating { 1249 d.mu.tableValidation.cond.Wait() 1250 } 1251 1252 var err error 1253 if n := len(d.mu.compact.inProgress); n > 0 { 1254 err = errors.Errorf("bitalostable: %d unexpected in-progress compactions", errors.Safe(n)) 1255 } 1256 err = firstError(err, d.mu.formatVers.marker.Close()) 1257 err = firstError(err, d.tableCache.close()) 1258 if !d.opts.ReadOnly { 1259 err = firstError(err, d.mu.log.Close()) 1260 } else if d.mu.log.LogWriter != nil { 1261 panic("bitalostable: log-writer should be nil in read-only mode") 1262 } 1263 err = firstError(err, d.fileLock.Close()) 1264 1265 // Note that versionSet.close() only closes the MANIFEST. The versions list 1266 // is still valid for the checks below. 1267 err = firstError(err, d.mu.versions.close()) 1268 1269 err = firstError(err, d.dataDir.Close()) 1270 if d.dataDir != d.walDir { 1271 err = firstError(err, d.walDir.Close()) 1272 } 1273 1274 d.readState.val.unrefLocked() 1275 1276 current := d.mu.versions.currentVersion() 1277 for v := d.mu.versions.versions.Front(); true; v = v.Next() { 1278 refs := v.Refs() 1279 if v == current { 1280 if refs != 1 { 1281 err = firstError(err, errors.Errorf("leaked iterators: current\n%s", v)) 1282 } 1283 break 1284 } 1285 if refs != 0 { 1286 err = firstError(err, errors.Errorf("leaked iterators:\n%s", v)) 1287 } 1288 } 1289 1290 for _, mem := range d.mu.mem.queue { 1291 mem.readerUnref() 1292 } 1293 if reserved := atomic.LoadInt64(&d.atomic.memTableReserved); reserved != 0 { 1294 err = firstError(err, errors.Errorf("leaked memtable reservation: %d", errors.Safe(reserved))) 1295 } 1296 1297 // No more cleaning can start. Wait for any async cleaning to complete. 1298 for d.mu.cleaner.cleaning { 1299 d.mu.cleaner.cond.Wait() 1300 } 1301 // There may still be obsolete tables if an existing async cleaning job 1302 // prevented a new cleaning job when a readState was unrefed. If needed, 1303 // synchronously delete obsolete files. 1304 if len(d.mu.versions.obsoleteTables) > 0 { 1305 d.deleteObsoleteFiles(d.mu.nextJobID, true /* waitForOngoing */) 1306 } 1307 // Wait for all the deletion goroutines spawned by cleaning jobs to finish. 1308 d.mu.Unlock() 1309 d.deleters.Wait() 1310 d.compactionSchedulers.Wait() 1311 d.mu.Lock() 1312 1313 // If the options include a closer to 'close' the filesystem, close it. 1314 if d.opts.private.fsCloser != nil { 1315 d.opts.private.fsCloser.Close() 1316 } 1317 return err 1318 } 1319 1320 func (d *DB) SetOptsDisableAutomaticCompactions(v bool) { 1321 if d.opts.DisableAutomaticCompactions != v { 1322 d.opts.DisableAutomaticCompactions = v 1323 } 1324 } 1325 1326 // Compact the specified range of keys in the database. 1327 func (d *DB) Compact(start, end []byte, parallelize bool) error { 1328 if err := d.closed.Load(); err != nil { 1329 panic(err) 1330 } 1331 if d.opts.ReadOnly { 1332 return ErrReadOnly 1333 } 1334 if d.cmp(start, end) >= 0 { 1335 return errors.Errorf("Compact start %s is not less than end %s", 1336 d.opts.Comparer.FormatKey(start), d.opts.Comparer.FormatKey(end)) 1337 } 1338 iStart := base.MakeInternalKey(start, InternalKeySeqNumMax, InternalKeyKindMax) 1339 iEnd := base.MakeInternalKey(end, 0, 0) 1340 m := (&fileMetadata{}).ExtendPointKeyBounds(d.cmp, iStart, iEnd) 1341 meta := []*fileMetadata{m} 1342 1343 d.mu.Lock() 1344 maxLevelWithFiles := 1 1345 cur := d.mu.versions.currentVersion() 1346 for level := 0; level < numLevels; level++ { 1347 overlaps := cur.Overlaps(level, d.cmp, start, end, iEnd.IsExclusiveSentinel()) 1348 if !overlaps.Empty() { 1349 maxLevelWithFiles = level + 1 1350 } 1351 } 1352 1353 // Determine if any memtable overlaps with the compaction range. We wait for 1354 // any such overlap to flush (initiating a flush if necessary). 1355 mem, err := func() (*flushableEntry, error) { 1356 // Check to see if any files overlap with any of the memtables. The queue 1357 // is ordered from oldest to newest with the mutable memtable being the 1358 // last element in the slice. We want to wait for the newest table that 1359 // overlaps. 1360 for i := len(d.mu.mem.queue) - 1; i >= 0; i-- { 1361 mem := d.mu.mem.queue[i] 1362 if ingestMemtableOverlaps(d.cmp, mem, meta) { 1363 var err error 1364 if mem.flushable == d.mu.mem.mutable { 1365 // We have to hold both commitPipeline.mu and DB.mu when calling 1366 // makeRoomForWrite(). Lock order requirements elsewhere force us to 1367 // unlock DB.mu in order to grab commitPipeline.mu first. 1368 d.mu.Unlock() 1369 d.commit.mu.Lock() 1370 d.mu.Lock() 1371 defer d.commit.mu.Unlock() 1372 if mem.flushable == d.mu.mem.mutable { 1373 // Only flush if the active memtable is unchanged. 1374 err = d.makeRoomForWrite(nil, true) 1375 } 1376 } 1377 mem.flushForced = true 1378 d.maybeScheduleFlush(true) 1379 return mem, err 1380 } 1381 } 1382 return nil, nil 1383 }() 1384 1385 d.mu.Unlock() 1386 1387 if err != nil { 1388 return err 1389 } 1390 if mem != nil { 1391 <-mem.flushed 1392 } 1393 1394 for level := 0; level < maxLevelWithFiles; { 1395 if err := d.manualCompact( 1396 iStart.UserKey, iEnd.UserKey, level, parallelize); err != nil { 1397 return err 1398 } 1399 level++ 1400 if level == numLevels-1 { 1401 // A manual compaction of the bottommost level occurred. 1402 // There is no next level to try and compact. 1403 break 1404 } 1405 } 1406 return nil 1407 } 1408 1409 func (d *DB) manualCompact(start, end []byte, level int, parallelize bool) error { 1410 d.mu.Lock() 1411 curr := d.mu.versions.currentVersion() 1412 files := curr.Overlaps(level, d.cmp, start, end, false) 1413 if files.Empty() { 1414 d.mu.Unlock() 1415 return nil 1416 } 1417 1418 var compactions []*manualCompaction 1419 if parallelize { 1420 compactions = append(compactions, d.splitManualCompaction(start, end, level)...) 1421 } else { 1422 compactions = append(compactions, &manualCompaction{ 1423 level: level, 1424 done: make(chan error, 1), 1425 start: start, 1426 end: end, 1427 }) 1428 } 1429 d.mu.compact.manual = append(d.mu.compact.manual, compactions...) 1430 d.maybeScheduleCompaction() 1431 d.mu.Unlock() 1432 1433 // Each of the channels is guaranteed to be eventually sent to once. After a 1434 // compaction is possibly picked in d.maybeScheduleCompaction(), either the 1435 // compaction is dropped, executed after being scheduled, or retried later. 1436 // Assuming eventual progress when a compaction is retried, all outcomes send 1437 // a value to the done channel. Since the channels are buffered, it is not 1438 // necessary to read from each channel, and so we can exit early in the event 1439 // of an error. 1440 for _, compaction := range compactions { 1441 if err := <-compaction.done; err != nil { 1442 return err 1443 } 1444 } 1445 return nil 1446 } 1447 1448 // splitManualCompaction splits a manual compaction over [start,end] on level 1449 // such that the resulting compactions have no key overlap. 1450 func (d *DB) splitManualCompaction( 1451 start, end []byte, level int, 1452 ) (splitCompactions []*manualCompaction) { 1453 curr := d.mu.versions.currentVersion() 1454 endLevel := level + 1 1455 baseLevel := d.mu.versions.picker.getBaseLevel() 1456 if level == 0 { 1457 endLevel = baseLevel 1458 } 1459 keyRanges := calculateInuseKeyRanges(curr, d.cmp, level, endLevel, start, end) 1460 for _, keyRange := range keyRanges { 1461 splitCompactions = append(splitCompactions, &manualCompaction{ 1462 level: level, 1463 done: make(chan error, 1), 1464 start: keyRange.Start, 1465 end: keyRange.End, 1466 split: true, 1467 }) 1468 } 1469 return splitCompactions 1470 } 1471 1472 // Flush the memtable to stable storage. 1473 func (d *DB) Flush() error { 1474 flushDone, err := d.AsyncFlush() 1475 if err != nil { 1476 return err 1477 } 1478 <-flushDone 1479 return nil 1480 } 1481 1482 // AsyncFlush asynchronously flushes the memtable to stable storage. 1483 // 1484 // If no error is returned, the caller can receive from the returned channel in 1485 // order to wait for the flush to complete. 1486 func (d *DB) AsyncFlush() (<-chan struct{}, error) { 1487 if err := d.closed.Load(); err != nil { 1488 panic(err) 1489 } 1490 if d.opts.ReadOnly { 1491 return nil, ErrReadOnly 1492 } 1493 1494 d.commit.mu.Lock() 1495 defer d.commit.mu.Unlock() 1496 d.mu.Lock() 1497 defer d.mu.Unlock() 1498 flushed := d.mu.mem.queue[len(d.mu.mem.queue)-1].flushed 1499 err := d.makeRoomForWrite(nil, false) 1500 if err != nil { 1501 return nil, err 1502 } 1503 return flushed, nil 1504 } 1505 1506 func (d *DB) Id() int { 1507 return d.opts.Id 1508 } 1509 1510 // InternalIntervalMetrics returns the InternalIntervalMetrics and resets for 1511 // the next interval (which is until the next call to this method). 1512 func (d *DB) InternalIntervalMetrics() *InternalIntervalMetrics { 1513 m := &InternalIntervalMetrics{} 1514 d.mu.Lock() 1515 defer d.mu.Unlock() 1516 if d.mu.log.metrics != nil { 1517 m.LogWriter.WriteThroughput = d.mu.log.metrics.WriteThroughput 1518 m.LogWriter.PendingBufferUtilization = 1519 d.mu.log.metrics.PendingBufferLen.Mean() / record.CapAllocatedBlocks 1520 m.LogWriter.SyncQueueUtilization = d.mu.log.metrics.SyncQueueLen.Mean() / record.SyncConcurrency 1521 m.LogWriter.SyncLatencyMicros = d.mu.log.metrics.SyncLatencyMicros 1522 d.mu.log.metrics = nil 1523 } 1524 m.Flush.WriteThroughput = d.mu.compact.flushWriteThroughput 1525 d.mu.compact.flushWriteThroughput = ThroughputMetric{} 1526 return m 1527 } 1528 1529 // Metrics returns metrics about the database. 1530 func (d *DB) Metrics() *Metrics { 1531 metrics := &Metrics{} 1532 recycledLogsCount, recycledLogSize := d.logRecycler.stats() 1533 1534 d.mu.Lock() 1535 vers := d.mu.versions.currentVersion() 1536 *metrics = d.mu.versions.metrics 1537 metrics.Compact.EstimatedDebt = d.mu.versions.picker.estimatedCompactionDebt(0) 1538 metrics.Compact.InProgressBytes = atomic.LoadInt64(&d.mu.versions.atomic.atomicInProgressBytes) 1539 metrics.Compact.NumInProgress = int64(d.mu.compact.compactingCount) 1540 metrics.Compact.MarkedFiles = vers.Stats.MarkedForCompaction 1541 for _, m := range d.mu.mem.queue { 1542 metrics.MemTable.Size += m.totalBytes() 1543 } 1544 metrics.Snapshots.Count = d.mu.snapshots.count() 1545 if metrics.Snapshots.Count > 0 { 1546 metrics.Snapshots.EarliestSeqNum = d.mu.snapshots.earliest() 1547 } 1548 metrics.MemTable.Count = int64(len(d.mu.mem.queue)) 1549 metrics.MemTable.ZombieCount = atomic.LoadInt64(&d.atomic.memTableCount) - metrics.MemTable.Count 1550 metrics.MemTable.ZombieSize = uint64(atomic.LoadInt64(&d.atomic.memTableReserved)) - metrics.MemTable.Size 1551 metrics.WAL.ObsoleteFiles = int64(recycledLogsCount) 1552 metrics.WAL.ObsoletePhysicalSize = recycledLogSize 1553 metrics.WAL.Size = atomic.LoadUint64(&d.atomic.logSize) 1554 // The current WAL size (d.atomic.logSize) is the current logical size, 1555 // which may be less than the WAL's physical size if it was recycled. 1556 // The file sizes in d.mu.log.queue are updated to the physical size 1557 // during WAL rotation. Use the larger of the two for the current WAL. All 1558 // the previous WALs's fileSizes in d.mu.log.queue are already updated. 1559 metrics.WAL.PhysicalSize = metrics.WAL.Size 1560 if len(d.mu.log.queue) > 0 && metrics.WAL.PhysicalSize < d.mu.log.queue[len(d.mu.log.queue)-1].fileSize { 1561 metrics.WAL.PhysicalSize = d.mu.log.queue[len(d.mu.log.queue)-1].fileSize 1562 } 1563 for i, n := 0, len(d.mu.log.queue)-1; i < n; i++ { 1564 metrics.WAL.PhysicalSize += d.mu.log.queue[i].fileSize 1565 } 1566 1567 metrics.WAL.BytesIn = d.mu.log.bytesIn // protected by d.mu 1568 for i, n := 0, len(d.mu.mem.queue)-1; i < n; i++ { 1569 metrics.WAL.Size += d.mu.mem.queue[i].logSize 1570 } 1571 metrics.WAL.BytesWritten = metrics.Levels[0].BytesIn + metrics.WAL.Size 1572 if p := d.mu.versions.picker; p != nil { 1573 compactions := d.getInProgressCompactionInfoLocked(nil) 1574 for level, score := range p.getScores(compactions) { 1575 metrics.Levels[level].Score = score 1576 } 1577 } 1578 metrics.Table.ZombieCount = int64(len(d.mu.versions.zombieTables)) 1579 for _, size := range d.mu.versions.zombieTables { 1580 metrics.Table.ZombieSize += size 1581 } 1582 metrics.private.optionsFileSize = d.optionsFileSize 1583 1584 metrics.Keys.RangeKeySetsCount = countRangeKeySetFragments(vers) 1585 1586 d.mu.versions.logLock() 1587 metrics.private.manifestFileSize = uint64(d.mu.versions.manifest.Size()) 1588 d.mu.versions.logUnlock() 1589 d.mu.Unlock() 1590 1591 metrics.BlockCache = d.opts.Cache.Metrics() 1592 metrics.TableCache, metrics.Filter = d.tableCache.metrics() 1593 metrics.TableIters = int64(d.tableCache.iterCount()) 1594 return metrics 1595 } 1596 1597 // sstablesOptions hold the optional parameters to retrieve TableInfo for all sstables. 1598 type sstablesOptions struct { 1599 // set to true will return the sstable properties in TableInfo 1600 withProperties bool 1601 } 1602 1603 // SSTablesOption set optional parameter used by `DB.SSTables`. 1604 type SSTablesOption func(*sstablesOptions) 1605 1606 // WithProperties enable return sstable properties in each TableInfo. 1607 // 1608 // NOTE: if most of the sstable properties need to be read from disk, 1609 // this options may make method `SSTables` quite slow. 1610 func WithProperties() SSTablesOption { 1611 return func(opt *sstablesOptions) { 1612 opt.withProperties = true 1613 } 1614 } 1615 1616 // SSTableInfo export manifest.TableInfo with sstable.Properties 1617 type SSTableInfo struct { 1618 manifest.TableInfo 1619 1620 // Properties is the sstable properties of this table. 1621 Properties *sstable.Properties 1622 } 1623 1624 // SSTables retrieves the current sstables. The returned slice is indexed by 1625 // level and each level is indexed by the position of the sstable within the 1626 // level. Note that this information may be out of date due to concurrent 1627 // flushes and compactions. 1628 func (d *DB) SSTables(opts ...SSTablesOption) ([][]SSTableInfo, error) { 1629 opt := &sstablesOptions{} 1630 for _, fn := range opts { 1631 fn(opt) 1632 } 1633 1634 // Grab and reference the current readState. 1635 readState := d.loadReadState() 1636 defer readState.unref() 1637 1638 // TODO(peter): This is somewhat expensive, especially on a large 1639 // database. It might be worthwhile to unify TableInfo and FileMetadata and 1640 // then we could simply return current.Files. Note that RocksDB is doing 1641 // something similar to the current code, so perhaps it isn't too bad. 1642 srcLevels := readState.current.Levels 1643 var totalTables int 1644 for i := range srcLevels { 1645 totalTables += srcLevels[i].Len() 1646 } 1647 1648 destTables := make([]SSTableInfo, totalTables) 1649 destLevels := make([][]SSTableInfo, len(srcLevels)) 1650 for i := range destLevels { 1651 iter := srcLevels[i].Iter() 1652 j := 0 1653 for m := iter.First(); m != nil; m = iter.Next() { 1654 destTables[j] = SSTableInfo{TableInfo: m.TableInfo()} 1655 if opt.withProperties { 1656 p, err := d.tableCache.getTableProperties(m) 1657 if err != nil { 1658 return nil, err 1659 } 1660 destTables[j].Properties = p 1661 } 1662 j++ 1663 } 1664 destLevels[i] = destTables[:j] 1665 destTables = destTables[j:] 1666 } 1667 return destLevels, nil 1668 } 1669 1670 // EstimateDiskUsage returns the estimated filesystem space used in bytes for 1671 // storing the range `[start, end]`. The estimation is computed as follows: 1672 // 1673 // - For sstables fully contained in the range the whole file size is included. 1674 // - For sstables partially contained in the range the overlapping data block sizes 1675 // are included. Even if a data block partially overlaps, or we cannot determine 1676 // overlap due to abbreviated index keys, the full data block size is included in 1677 // the estimation. Note that unlike fully contained sstables, none of the 1678 // meta-block space is counted for partially overlapped files. 1679 // - There may also exist WAL entries for unflushed keys in this range. This 1680 // estimation currently excludes space used for the range in the WAL. 1681 func (d *DB) EstimateDiskUsage(start, end []byte) (uint64, error) { 1682 if err := d.closed.Load(); err != nil { 1683 panic(err) 1684 } 1685 if d.opts.Comparer.Compare(start, end) > 0 { 1686 return 0, errors.New("invalid key-range specified (start > end)") 1687 } 1688 1689 // Grab and reference the current readState. This prevents the underlying 1690 // files in the associated version from being deleted if there is a concurrent 1691 // compaction. 1692 readState := d.loadReadState() 1693 defer readState.unref() 1694 1695 var totalSize uint64 1696 for level, files := range readState.current.Levels { 1697 iter := files.Iter() 1698 if level > 0 { 1699 // We can only use `Overlaps` to restrict `files` at L1+ since at L0 it 1700 // expands the range iteratively until it has found a set of files that 1701 // do not overlap any other L0 files outside that set. 1702 overlaps := readState.current.Overlaps(level, d.opts.Comparer.Compare, start, end, false /* exclusiveEnd */) 1703 iter = overlaps.Iter() 1704 } 1705 for file := iter.First(); file != nil; file = iter.Next() { 1706 if d.opts.Comparer.Compare(start, file.Smallest.UserKey) <= 0 && 1707 d.opts.Comparer.Compare(file.Largest.UserKey, end) <= 0 { 1708 // The range fully contains the file, so skip looking it up in 1709 // table cache/looking at its indexes, and add the full file size. 1710 totalSize += file.Size 1711 } else if d.opts.Comparer.Compare(file.Smallest.UserKey, end) <= 0 && 1712 d.opts.Comparer.Compare(start, file.Largest.UserKey) <= 0 { 1713 var size uint64 1714 err := d.tableCache.withReader(file, func(r *sstable.Reader) (err error) { 1715 size, err = r.EstimateDiskUsage(start, end) 1716 return err 1717 }) 1718 if err != nil { 1719 return 0, err 1720 } 1721 totalSize += size 1722 } 1723 } 1724 } 1725 return totalSize, nil 1726 } 1727 1728 func (d *DB) walPreallocateSize() int { 1729 // Set the WAL preallocate size to 110% of the memtable size. Note that there 1730 // is a bit of apples and oranges in units here as the memtabls size 1731 // corresponds to the memory usage of the memtable while the WAL size is the 1732 // size of the batches (plus overhead) stored in the WAL. 1733 // 1734 // TODO(peter): 110% of the memtable size is quite hefty for a block 1735 // size. This logic is taken from GetWalPreallocateBlockSize in 1736 // RocksDB. Could a smaller preallocation block size be used? 1737 size := d.opts.MemTableSize 1738 size = (size / 10) + size 1739 return size 1740 } 1741 1742 func (d *DB) newMemTable(logNum FileNum, logSeqNum uint64) (*memTable, *flushableEntry) { 1743 size := d.mu.mem.nextSize 1744 if d.mu.mem.nextSize < d.opts.MemTableSize { 1745 d.mu.mem.nextSize *= 2 1746 if d.mu.mem.nextSize > d.opts.MemTableSize { 1747 d.mu.mem.nextSize = d.opts.MemTableSize 1748 } 1749 } 1750 1751 atomic.AddInt64(&d.atomic.memTableCount, 1) 1752 atomic.AddInt64(&d.atomic.memTableReserved, int64(size)) 1753 releaseAccountingReservation := d.opts.Cache.Reserve(size) 1754 1755 mem := newMemTable(memTableOptions{ 1756 Options: d.opts, 1757 arenaBuf: manual.New(int(size)), 1758 logSeqNum: logSeqNum, 1759 }) 1760 1761 // Note: this is a no-op if invariants are disabled or race is enabled. 1762 invariants.SetFinalizer(mem, checkMemTable) 1763 1764 entry := d.newFlushableEntry(mem, logNum, logSeqNum) 1765 entry.releaseMemAccounting = func() { 1766 manual.Free(mem.arenaBuf) 1767 mem.arenaBuf = nil 1768 atomic.AddInt64(&d.atomic.memTableCount, -1) 1769 atomic.AddInt64(&d.atomic.memTableReserved, -int64(size)) 1770 releaseAccountingReservation() 1771 } 1772 return mem, entry 1773 } 1774 1775 func (d *DB) newFlushableEntry(f flushable, logNum FileNum, logSeqNum uint64) *flushableEntry { 1776 return &flushableEntry{ 1777 flushable: f, 1778 flushed: make(chan struct{}), 1779 logNum: logNum, 1780 logSeqNum: logSeqNum, 1781 readerRefs: 1, 1782 } 1783 } 1784 1785 // makeRoomForWrite ensures that the memtable has room to hold the contents of 1786 // Batch. It reserves the space in the memtable and adds a reference to the 1787 // memtable. The caller must later ensure that the memtable is unreferenced. If 1788 // the memtable is full, or a nil Batch is provided, the current memtable is 1789 // rotated (marked as immutable) and a new mutable memtable is allocated. This 1790 // memtable rotation also causes a log rotation. 1791 // 1792 // Both DB.mu and commitPipeline.mu must be held by the caller. Note that DB.mu 1793 // may be released and reacquired. 1794 func (d *DB) makeRoomForWrite(b *Batch, needReport bool) error { 1795 force := b == nil || b.flushable != nil 1796 isFlushBatch := b != nil && b.flushable != nil && b.isFlush 1797 stalled := false 1798 for { 1799 if d.mu.mem.switching { 1800 d.mu.mem.cond.Wait() 1801 continue 1802 } 1803 if b != nil && b.flushable == nil { 1804 err := d.mu.mem.mutable.prepare(b) 1805 if err != arenaskl.ErrArenaFull { 1806 if stalled { 1807 d.opts.EventListener.WriteStallEnd() 1808 } 1809 return err 1810 } 1811 } else if !force { 1812 if stalled { 1813 d.opts.EventListener.WriteStallEnd() 1814 } 1815 return nil 1816 } 1817 // force || err == ErrArenaFull, so we need to rotate the current memtable. 1818 if !isFlushBatch { 1819 var size uint64 1820 for i := range d.mu.mem.queue { 1821 size += d.mu.mem.queue[i].totalBytes() 1822 } 1823 if size >= uint64(d.opts.MemTableStopWritesThreshold)*uint64(d.opts.MemTableSize) { 1824 // We have filled up the current memtable, but already queued memtables 1825 // are still flushing, so we wait. 1826 if !stalled { 1827 stalled = true 1828 d.opts.EventListener.WriteStallBegin(WriteStallBeginInfo{ 1829 Reason: "memtable count limit reached", 1830 }) 1831 } 1832 d.mu.compact.cond.Wait() 1833 continue 1834 } 1835 } 1836 1837 l0ReadAmp := d.mu.versions.currentVersion().L0Sublevels.ReadAmplification() 1838 if l0ReadAmp >= d.opts.L0StopWritesThreshold { 1839 // There are too many level-0 files, so we wait. 1840 if !stalled { 1841 stalled = true 1842 d.opts.EventListener.WriteStallBegin(WriteStallBeginInfo{ 1843 Reason: "L0 file count limit exceeded", 1844 }) 1845 } 1846 d.mu.compact.cond.Wait() 1847 continue 1848 } 1849 1850 var newLogNum FileNum 1851 var newLogFile vfs.File 1852 var newLogSize uint64 1853 var prevLogSize uint64 1854 var err error 1855 1856 if !d.opts.DisableWAL { 1857 jobID := d.mu.nextJobID 1858 d.mu.nextJobID++ 1859 newLogNum = d.mu.versions.getNextFileNum() 1860 d.mu.mem.switching = true 1861 1862 prevLogSize = uint64(d.mu.log.Size()) 1863 1864 // The previous log may have grown past its original physical 1865 // size. Update its file size in the queue so we have a proper 1866 // accounting of its file size. 1867 if d.mu.log.queue[len(d.mu.log.queue)-1].fileSize < prevLogSize { 1868 d.mu.log.queue[len(d.mu.log.queue)-1].fileSize = prevLogSize 1869 } 1870 d.mu.Unlock() 1871 1872 // Close the previous log first. This writes an EOF trailer 1873 // signifying the end of the file and syncs it to disk. We must 1874 // close the previous log before linking the new log file, 1875 // otherwise a crash could leave both logs with unclean tails, and 1876 // Open will treat the previous log as corrupt. 1877 err = d.mu.log.LogWriter.Close() 1878 metrics := d.mu.log.LogWriter.Metrics() 1879 d.mu.Lock() 1880 if d.mu.log.metrics == nil { 1881 d.mu.log.metrics = metrics 1882 } else { 1883 if err := d.mu.log.metrics.Merge(metrics); err != nil { 1884 d.opts.Logger.Infof("metrics error: %s", err) 1885 } 1886 } 1887 d.mu.Unlock() 1888 1889 newLogName := base.MakeFilepath(d.opts.FS, d.walDirname, fileTypeLog, newLogNum) 1890 1891 // Try to use a recycled log file. Recycling log files is an important 1892 // performance optimization as it is faster to sync a file that has 1893 // already been written, than one which is being written for the first 1894 // time. This is due to the need to sync file metadata when a file is 1895 // being written for the first time. Note this is true even if file 1896 // preallocation is performed (e.g. fallocate). 1897 var recycleLog fileInfo 1898 var recycleOK bool 1899 if err == nil { 1900 recycleLog, recycleOK = d.logRecycler.peek() 1901 if recycleOK { 1902 recycleLogName := base.MakeFilepath(d.opts.FS, d.walDirname, fileTypeLog, recycleLog.fileNum) 1903 newLogFile, err = d.opts.FS.ReuseForWrite(recycleLogName, newLogName) 1904 base.MustExist(d.opts.FS, newLogName, d.opts.Logger, err) 1905 } else { 1906 newLogFile, err = d.opts.FS.Create(newLogName) 1907 base.MustExist(d.opts.FS, newLogName, d.opts.Logger, err) 1908 } 1909 } 1910 1911 if err == nil && recycleOK { 1912 // Figure out the recycled WAL size. This Stat is necessary 1913 // because ReuseForWrite's contract allows for removing the 1914 // old file and creating a new one. We don't know whether the 1915 // WAL was actually recycled. 1916 // TODO(jackson): Adding a boolean to the ReuseForWrite return 1917 // value indicating whether or not the file was actually 1918 // reused would allow us to skip the stat and use 1919 // recycleLog.fileSize. 1920 var finfo os.FileInfo 1921 finfo, err = newLogFile.Stat() 1922 if err == nil { 1923 newLogSize = uint64(finfo.Size()) 1924 } 1925 } 1926 1927 if err == nil { 1928 // TODO(peter): RocksDB delays sync of the parent directory until the 1929 // first time the log is synced. Is that worthwhile? 1930 err = d.walDir.Sync() 1931 } 1932 1933 if err != nil && newLogFile != nil { 1934 newLogFile.Close() 1935 } else if err == nil { 1936 newLogFile = vfs.NewSyncingFile(newLogFile, vfs.SyncingFileOptions{ 1937 NoSyncOnClose: d.opts.NoSyncOnClose, 1938 BytesPerSync: d.opts.WALBytesPerSync, 1939 PreallocateSize: d.walPreallocateSize(), 1940 }) 1941 } 1942 1943 if recycleOK { 1944 err = firstError(err, d.logRecycler.pop(recycleLog.fileNum)) 1945 } 1946 1947 d.opts.EventListener.WALCreated(WALCreateInfo{ 1948 JobID: jobID, 1949 Path: newLogName, 1950 FileNum: newLogNum, 1951 RecycledFileNum: recycleLog.fileNum, 1952 Err: err, 1953 }) 1954 1955 d.mu.Lock() 1956 d.mu.mem.switching = false 1957 d.mu.mem.cond.Broadcast() 1958 1959 d.mu.versions.metrics.WAL.Files++ 1960 } 1961 1962 if err != nil { 1963 // TODO(peter): avoid chewing through file numbers in a tight loop if there 1964 // is an error here. 1965 // 1966 // What to do here? Stumbling on doesn't seem worthwhile. If we failed to 1967 // close the previous log it is possible we lost a write. 1968 panic(err) 1969 } 1970 1971 if !d.opts.DisableWAL { 1972 d.mu.log.queue = append(d.mu.log.queue, fileInfo{fileNum: newLogNum, fileSize: newLogSize}) 1973 d.mu.log.LogWriter = record.NewLogWriter(newLogFile, newLogNum) 1974 d.mu.log.LogWriter.SetMinSyncInterval(d.opts.WALMinSyncInterval) 1975 } 1976 1977 immMem := d.mu.mem.mutable 1978 imm := d.mu.mem.queue[len(d.mu.mem.queue)-1] 1979 imm.logSize = prevLogSize 1980 imm.flushForced = imm.flushForced || (b == nil) 1981 1982 // If we are manually flushing and we used less than half of the bytes in 1983 // the memtable, don't increase the size for the next memtable. This 1984 // reduces memtable memory pressure when an application is frequently 1985 // manually flushing. 1986 if (b == nil) && uint64(immMem.availBytes()) > immMem.totalBytes()/2 { 1987 d.mu.mem.nextSize = int(immMem.totalBytes()) 1988 } 1989 1990 if b != nil && b.flushable != nil { 1991 // The batch is too large to fit in the memtable so add it directly to 1992 // the immutable queue. The flushable batch is associated with the same 1993 // log as the immutable memtable, but logically occurs after it in 1994 // seqnum space. So give the flushable batch the logNum and clear it from 1995 // the immutable log. This is done as a defensive measure to prevent the 1996 // WAL containing the large batch from being deleted prematurely if the 1997 // corresponding memtable is flushed without flushing the large batch. 1998 // 1999 // See DB.commitWrite for the special handling of log writes for large 2000 // batches. In particular, the large batch has already written to 2001 // imm.logNum. 2002 entry := d.newFlushableEntry(b.flushable, imm.logNum, b.SeqNum()) 2003 // The large batch is by definition large. Reserve space from the cache 2004 // for it until it is flushed. 2005 entry.releaseMemAccounting = d.opts.Cache.Reserve(int(b.flushable.totalBytes())) 2006 if isFlushBatch { 2007 entry.alloc = b.alloc 2008 } 2009 2010 d.mu.mem.queue = append(d.mu.mem.queue, entry) 2011 imm.logNum = 0 2012 } 2013 2014 var logSeqNum uint64 2015 if b != nil { 2016 logSeqNum = b.SeqNum() 2017 if b.flushable != nil { 2018 logSeqNum += uint64(b.Count()) 2019 } 2020 } else { 2021 logSeqNum = atomic.LoadUint64(&d.mu.versions.atomic.logSeqNum) 2022 } 2023 2024 // Create a new memtable, scheduling the previous one for flushing. We do 2025 // this even if the previous memtable was empty because the DB.Flush 2026 // mechanism is dependent on being able to wait for the empty memtable to 2027 // flush. We can't just mark the empty memtable as flushed here because we 2028 // also have to wait for all previous immutable tables to 2029 // flush. Additionally, the memtable is tied to particular WAL file and we 2030 // want to go through the flush path in order to recycle that WAL file. 2031 // 2032 // NB: newLogNum corresponds to the WAL that contains mutations that are 2033 // present in the new memtable. When immutable memtables are flushed to 2034 // disk, a VersionEdit will be created telling the manifest the minimum 2035 // unflushed log number (which will be the next one in d.mu.mem.mutable 2036 // that was not flushed). 2037 var entry *flushableEntry 2038 d.mu.mem.mutable, entry = d.newMemTable(newLogNum, logSeqNum) 2039 d.mu.mem.queue = append(d.mu.mem.queue, entry) 2040 d.updateReadStateLocked(nil) 2041 if immMem.writerUnref() { 2042 d.maybeScheduleFlush(needReport) 2043 } 2044 force = false 2045 } 2046 } 2047 2048 func (d *DB) getEarliestUnflushedSeqNumLocked() uint64 { 2049 seqNum := InternalKeySeqNumMax 2050 for i := range d.mu.mem.queue { 2051 logSeqNum := d.mu.mem.queue[i].logSeqNum 2052 if seqNum > logSeqNum { 2053 seqNum = logSeqNum 2054 } 2055 } 2056 return seqNum 2057 } 2058 2059 func (d *DB) getInProgressCompactionInfoLocked(finishing *compaction) (rv []compactionInfo) { 2060 for c := range d.mu.compact.inProgress { 2061 if len(c.flushing) == 0 && (finishing == nil || c != finishing) { 2062 info := compactionInfo{ 2063 inputs: c.inputs, 2064 smallest: c.smallest, 2065 largest: c.largest, 2066 outputLevel: -1, 2067 } 2068 if c.outputLevel != nil { 2069 info.outputLevel = c.outputLevel.level 2070 } 2071 rv = append(rv, info) 2072 } 2073 } 2074 return 2075 } 2076 2077 func inProgressL0Compactions(inProgress []compactionInfo) []manifest.L0Compaction { 2078 var compactions []manifest.L0Compaction 2079 for _, info := range inProgress { 2080 l0 := false 2081 for _, cl := range info.inputs { 2082 l0 = l0 || cl.level == 0 2083 } 2084 if !l0 { 2085 continue 2086 } 2087 compactions = append(compactions, manifest.L0Compaction{ 2088 Smallest: info.smallest, 2089 Largest: info.largest, 2090 IsIntraL0: info.outputLevel == 0, 2091 }) 2092 } 2093 return compactions 2094 } 2095 2096 // firstError returns the first non-nil error of err0 and err1, or nil if both 2097 // are nil. 2098 func firstError(err0, err1 error) error { 2099 if err0 != nil { 2100 return err0 2101 } 2102 return err1 2103 }