github.com/cockroachdb/pebble@v1.1.2/db.go (about) 1 // Copyright 2012 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 pebble provides an ordered key/value store. 6 package pebble // import "github.com/cockroachdb/pebble" 7 8 import ( 9 "context" 10 "fmt" 11 "io" 12 "os" 13 "strconv" 14 "sync" 15 "sync/atomic" 16 "time" 17 18 "github.com/cockroachdb/errors" 19 "github.com/cockroachdb/pebble/internal/arenaskl" 20 "github.com/cockroachdb/pebble/internal/base" 21 "github.com/cockroachdb/pebble/internal/invalidating" 22 "github.com/cockroachdb/pebble/internal/invariants" 23 "github.com/cockroachdb/pebble/internal/keyspan" 24 "github.com/cockroachdb/pebble/internal/manifest" 25 "github.com/cockroachdb/pebble/internal/manual" 26 "github.com/cockroachdb/pebble/objstorage" 27 "github.com/cockroachdb/pebble/objstorage/remote" 28 "github.com/cockroachdb/pebble/rangekey" 29 "github.com/cockroachdb/pebble/record" 30 "github.com/cockroachdb/pebble/sstable" 31 "github.com/cockroachdb/pebble/vfs" 32 "github.com/cockroachdb/pebble/vfs/atomicfs" 33 "github.com/cockroachdb/tokenbucket" 34 "github.com/prometheus/client_golang/prometheus" 35 ) 36 37 const ( 38 // minTableCacheSize is the minimum size of the table cache, for a single db. 39 minTableCacheSize = 64 40 41 // numNonTableCacheFiles is an approximation for the number of files 42 // that we don't use for table caches, for a given db. 43 numNonTableCacheFiles = 10 44 ) 45 46 var ( 47 // ErrNotFound is returned when a get operation does not find the requested 48 // key. 49 ErrNotFound = base.ErrNotFound 50 // ErrClosed is panicked when an operation is performed on a closed snapshot or 51 // DB. Use errors.Is(err, ErrClosed) to check for this error. 52 ErrClosed = errors.New("pebble: closed") 53 // ErrReadOnly is returned when a write operation is performed on a read-only 54 // database. 55 ErrReadOnly = errors.New("pebble: read-only") 56 // errNoSplit indicates that the user is trying to perform a range key 57 // operation but the configured Comparer does not provide a Split 58 // implementation. 59 errNoSplit = errors.New("pebble: Comparer.Split required for range key operations") 60 ) 61 62 // Reader is a readable key/value store. 63 // 64 // It is safe to call Get and NewIter from concurrent goroutines. 65 type Reader interface { 66 // Get gets the value for the given key. It returns ErrNotFound if the DB 67 // does not contain the key. 68 // 69 // The caller should not modify the contents of the returned slice, but it is 70 // safe to modify the contents of the argument after Get returns. The 71 // returned slice will remain valid until the returned Closer is closed. On 72 // success, the caller MUST call closer.Close() or a memory leak will occur. 73 Get(key []byte) (value []byte, closer io.Closer, err error) 74 75 // NewIter returns an iterator that is unpositioned (Iterator.Valid() will 76 // return false). The iterator can be positioned via a call to SeekGE, 77 // SeekLT, First or Last. 78 NewIter(o *IterOptions) (*Iterator, error) 79 80 // Close closes the Reader. It may or may not close any underlying io.Reader 81 // or io.Writer, depending on how the DB was created. 82 // 83 // It is not safe to close a DB until all outstanding iterators are closed. 84 // It is valid to call Close multiple times. Other methods should not be 85 // called after the DB has been closed. 86 Close() error 87 } 88 89 // Writer is a writable key/value store. 90 // 91 // Goroutine safety is dependent on the specific implementation. 92 type Writer interface { 93 // Apply the operations contained in the batch to the DB. 94 // 95 // It is safe to modify the contents of the arguments after Apply returns. 96 Apply(batch *Batch, o *WriteOptions) error 97 98 // Delete deletes the value for the given key. Deletes are blind all will 99 // succeed even if the given key does not exist. 100 // 101 // It is safe to modify the contents of the arguments after Delete returns. 102 Delete(key []byte, o *WriteOptions) error 103 104 // DeleteSized behaves identically to Delete, but takes an additional 105 // argument indicating the size of the value being deleted. DeleteSized 106 // should be preferred when the caller has the expectation that there exists 107 // a single internal KV pair for the key (eg, the key has not been 108 // overwritten recently), and the caller knows the size of its value. 109 // 110 // DeleteSized will record the value size within the tombstone and use it to 111 // inform compaction-picking heuristics which strive to reduce space 112 // amplification in the LSM. This "calling your shot" mechanic allows the 113 // storage engine to more accurately estimate and reduce space 114 // amplification. 115 // 116 // It is safe to modify the contents of the arguments after DeleteSized 117 // returns. 118 DeleteSized(key []byte, valueSize uint32, _ *WriteOptions) error 119 120 // SingleDelete is similar to Delete in that it deletes the value for the given key. Like Delete, 121 // it is a blind operation that will succeed even if the given key does not exist. 122 // 123 // WARNING: Undefined (non-deterministic) behavior will result if a key is overwritten and 124 // then deleted using SingleDelete. The record may appear deleted immediately, but be 125 // resurrected at a later time after compactions have been performed. Or the record may 126 // be deleted permanently. A Delete operation lays down a "tombstone" which shadows all 127 // previous versions of a key. The SingleDelete operation is akin to "anti-matter" and will 128 // only delete the most recently written version for a key. These different semantics allow 129 // the DB to avoid propagating a SingleDelete operation during a compaction as soon as the 130 // corresponding Set operation is encountered. These semantics require extreme care to handle 131 // properly. Only use if you have a workload where the performance gain is critical and you 132 // can guarantee that a record is written once and then deleted once. 133 // 134 // SingleDelete is internally transformed into a Delete if the most recent record for a key is either 135 // a Merge or Delete record. 136 // 137 // It is safe to modify the contents of the arguments after SingleDelete returns. 138 SingleDelete(key []byte, o *WriteOptions) error 139 140 // DeleteRange deletes all of the point keys (and values) in the range 141 // [start,end) (inclusive on start, exclusive on end). DeleteRange does NOT 142 // delete overlapping range keys (eg, keys set via RangeKeySet). 143 // 144 // It is safe to modify the contents of the arguments after DeleteRange 145 // returns. 146 DeleteRange(start, end []byte, o *WriteOptions) error 147 148 // LogData adds the specified to the batch. The data will be written to the 149 // WAL, but not added to memtables or sstables. Log data is never indexed, 150 // which makes it useful for testing WAL performance. 151 // 152 // It is safe to modify the contents of the argument after LogData returns. 153 LogData(data []byte, opts *WriteOptions) error 154 155 // Merge merges the value for the given key. The details of the merge are 156 // dependent upon the configured merge operation. 157 // 158 // It is safe to modify the contents of the arguments after Merge returns. 159 Merge(key, value []byte, o *WriteOptions) error 160 161 // Set sets the value for the given key. It overwrites any previous value 162 // for that key; a DB is not a multi-map. 163 // 164 // It is safe to modify the contents of the arguments after Set returns. 165 Set(key, value []byte, o *WriteOptions) error 166 167 // RangeKeySet sets a range key mapping the key range [start, end) at the MVCC 168 // timestamp suffix to value. The suffix is optional. If any portion of the key 169 // range [start, end) is already set by a range key with the same suffix value, 170 // RangeKeySet overrides it. 171 // 172 // It is safe to modify the contents of the arguments after RangeKeySet returns. 173 RangeKeySet(start, end, suffix, value []byte, opts *WriteOptions) error 174 175 // RangeKeyUnset removes a range key mapping the key range [start, end) at the 176 // MVCC timestamp suffix. The suffix may be omitted to remove an unsuffixed 177 // range key. RangeKeyUnset only removes portions of range keys that fall within 178 // the [start, end) key span, and only range keys with suffixes that exactly 179 // match the unset suffix. 180 // 181 // It is safe to modify the contents of the arguments after RangeKeyUnset 182 // returns. 183 RangeKeyUnset(start, end, suffix []byte, opts *WriteOptions) error 184 185 // RangeKeyDelete deletes all of the range keys in the range [start,end) 186 // (inclusive on start, exclusive on end). It does not delete point keys (for 187 // that use DeleteRange). RangeKeyDelete removes all range keys within the 188 // bounds, including those with or without suffixes. 189 // 190 // It is safe to modify the contents of the arguments after RangeKeyDelete 191 // returns. 192 RangeKeyDelete(start, end []byte, opts *WriteOptions) error 193 } 194 195 // CPUWorkHandle represents a handle used by the CPUWorkPermissionGranter API. 196 type CPUWorkHandle interface { 197 // Permitted indicates whether Pebble can use additional CPU resources. 198 Permitted() bool 199 } 200 201 // CPUWorkPermissionGranter is used to request permission to opportunistically 202 // use additional CPUs to speed up internal background work. 203 type CPUWorkPermissionGranter interface { 204 // GetPermission returns a handle regardless of whether permission is granted 205 // or not. In the latter case, the handle is only useful for recording 206 // the CPU time actually spent on this calling goroutine. 207 GetPermission(time.Duration) CPUWorkHandle 208 // CPUWorkDone must be called regardless of whether CPUWorkHandle.Permitted 209 // returns true or false. 210 CPUWorkDone(CPUWorkHandle) 211 } 212 213 // Use a default implementation for the CPU work granter to avoid excessive nil 214 // checks in the code. 215 type defaultCPUWorkHandle struct{} 216 217 func (d defaultCPUWorkHandle) Permitted() bool { 218 return false 219 } 220 221 type defaultCPUWorkGranter struct{} 222 223 func (d defaultCPUWorkGranter) GetPermission(_ time.Duration) CPUWorkHandle { 224 return defaultCPUWorkHandle{} 225 } 226 227 func (d defaultCPUWorkGranter) CPUWorkDone(_ CPUWorkHandle) {} 228 229 // DB provides a concurrent, persistent ordered key/value store. 230 // 231 // A DB's basic operations (Get, Set, Delete) should be self-explanatory. Get 232 // and Delete will return ErrNotFound if the requested key is not in the store. 233 // Callers are free to ignore this error. 234 // 235 // A DB also allows for iterating over the key/value pairs in key order. If d 236 // is a DB, the code below prints all key/value pairs whose keys are 'greater 237 // than or equal to' k: 238 // 239 // iter := d.NewIter(readOptions) 240 // for iter.SeekGE(k); iter.Valid(); iter.Next() { 241 // fmt.Printf("key=%q value=%q\n", iter.Key(), iter.Value()) 242 // } 243 // return iter.Close() 244 // 245 // The Options struct holds the optional parameters for the DB, including a 246 // Comparer to define a 'less than' relationship over keys. It is always valid 247 // to pass a nil *Options, which means to use the default parameter values. Any 248 // zero field of a non-nil *Options also means to use the default value for 249 // that parameter. Thus, the code below uses a custom Comparer, but the default 250 // values for every other parameter: 251 // 252 // db := pebble.Open(&Options{ 253 // Comparer: myComparer, 254 // }) 255 type DB struct { 256 // The count and size of referenced memtables. This includes memtables 257 // present in DB.mu.mem.queue, as well as memtables that have been flushed 258 // but are still referenced by an inuse readState, as well as up to one 259 // memTable waiting to be reused and stored in d.memTableRecycle. 260 memTableCount atomic.Int64 261 memTableReserved atomic.Int64 // number of bytes reserved in the cache for memtables 262 // memTableRecycle holds a pointer to an obsolete memtable. The next 263 // memtable allocation will reuse this memtable if it has not already been 264 // recycled. 265 memTableRecycle atomic.Pointer[memTable] 266 267 // The size of the current log file (i.e. db.mu.log.queue[len(queue)-1]. 268 logSize atomic.Uint64 269 270 // The number of bytes available on disk. 271 diskAvailBytes atomic.Uint64 272 273 cacheID uint64 274 dirname string 275 walDirname string 276 opts *Options 277 cmp Compare 278 equal Equal 279 merge Merge 280 split Split 281 abbreviatedKey AbbreviatedKey 282 // The threshold for determining when a batch is "large" and will skip being 283 // inserted into a memtable. 284 largeBatchThreshold uint64 285 // The current OPTIONS file number. 286 optionsFileNum base.DiskFileNum 287 // The on-disk size of the current OPTIONS file. 288 optionsFileSize uint64 289 290 // objProvider is used to access and manage SSTs. 291 objProvider objstorage.Provider 292 293 fileLock *Lock 294 dataDir vfs.File 295 walDir vfs.File 296 297 tableCache *tableCacheContainer 298 newIters tableNewIters 299 tableNewRangeKeyIter keyspan.TableNewSpanIter 300 301 commit *commitPipeline 302 303 // readState provides access to the state needed for reading without needing 304 // to acquire DB.mu. 305 readState struct { 306 sync.RWMutex 307 val *readState 308 } 309 // logRecycler holds a set of log file numbers that are available for 310 // reuse. Writing to a recycled log file is faster than to a new log file on 311 // some common filesystems (xfs, and ext3/4) due to avoiding metadata 312 // updates. 313 logRecycler logRecycler 314 315 closed *atomic.Value 316 closedCh chan struct{} 317 318 cleanupManager *cleanupManager 319 320 // During an iterator close, we may asynchronously schedule read compactions. 321 // We want to wait for those goroutines to finish, before closing the DB. 322 // compactionShedulers.Wait() should not be called while the DB.mu is held. 323 compactionSchedulers sync.WaitGroup 324 325 // The main mutex protecting internal DB state. This mutex encompasses many 326 // fields because those fields need to be accessed and updated atomically. In 327 // particular, the current version, log.*, mem.*, and snapshot list need to 328 // be accessed and updated atomically during compaction. 329 // 330 // Care is taken to avoid holding DB.mu during IO operations. Accomplishing 331 // this sometimes requires releasing DB.mu in a method that was called with 332 // it held. See versionSet.logAndApply() and DB.makeRoomForWrite() for 333 // examples. This is a common pattern, so be careful about expectations that 334 // DB.mu will be held continuously across a set of calls. 335 mu struct { 336 sync.Mutex 337 338 formatVers struct { 339 // vers is the database's current format major version. 340 // Backwards-incompatible features are gated behind new 341 // format major versions and not enabled until a database's 342 // version is ratcheted upwards. 343 // 344 // Although this is under the `mu` prefix, readers may read vers 345 // atomically without holding d.mu. Writers must only write to this 346 // value through finalizeFormatVersUpgrade which requires d.mu is 347 // held. 348 vers atomic.Uint64 349 // marker is the atomic marker for the format major version. 350 // When a database's version is ratcheted upwards, the 351 // marker is moved in order to atomically record the new 352 // version. 353 marker *atomicfs.Marker 354 // ratcheting when set to true indicates that the database is 355 // currently in the process of ratcheting the format major version 356 // to vers + 1. As a part of ratcheting the format major version, 357 // migrations may drop and re-acquire the mutex. 358 ratcheting bool 359 } 360 361 // The ID of the next job. Job IDs are passed to event listener 362 // notifications and act as a mechanism for tying together the events and 363 // log messages for a single job such as a flush, compaction, or file 364 // ingestion. Job IDs are not serialized to disk or used for correctness. 365 nextJobID int 366 367 // The collection of immutable versions and state about the log and visible 368 // sequence numbers. Use the pointer here to ensure the atomic fields in 369 // version set are aligned properly. 370 versions *versionSet 371 372 log struct { 373 // The queue of logs, containing both flushed and unflushed logs. The 374 // flushed logs will be a prefix, the unflushed logs a suffix. The 375 // delimeter between flushed and unflushed logs is 376 // versionSet.minUnflushedLogNum. 377 queue []fileInfo 378 // The number of input bytes to the log. This is the raw size of the 379 // batches written to the WAL, without the overhead of the record 380 // envelopes. Requires DB.mu to be held when read or written. 381 bytesIn uint64 382 // The LogWriter is protected by commitPipeline.mu. This allows log 383 // writes to be performed without holding DB.mu, but requires both 384 // commitPipeline.mu and DB.mu to be held when rotating the WAL/memtable 385 // (i.e. makeRoomForWrite). 386 *record.LogWriter 387 // Can be nil. 388 metrics struct { 389 fsyncLatency prometheus.Histogram 390 record.LogWriterMetrics 391 } 392 registerLogWriterForTesting func(w *record.LogWriter) 393 } 394 395 mem struct { 396 // The current mutable memTable. 397 mutable *memTable 398 // Queue of flushables (the mutable memtable is at end). Elements are 399 // added to the end of the slice and removed from the beginning. Once an 400 // index is set it is never modified making a fixed slice immutable and 401 // safe for concurrent reads. 402 queue flushableList 403 // nextSize is the size of the next memtable. The memtable size starts at 404 // min(256KB,Options.MemTableSize) and doubles each time a new memtable 405 // is allocated up to Options.MemTableSize. This reduces the memory 406 // footprint of memtables when lots of DB instances are used concurrently 407 // in test environments. 408 nextSize uint64 409 } 410 411 compact struct { 412 // Condition variable used to signal when a flush or compaction has 413 // completed. Used by the write-stall mechanism to wait for the stall 414 // condition to clear. See DB.makeRoomForWrite(). 415 cond sync.Cond 416 // True when a flush is in progress. 417 flushing bool 418 // The number of ongoing compactions. 419 compactingCount int 420 // The list of deletion hints, suggesting ranges for delete-only 421 // compactions. 422 deletionHints []deleteCompactionHint 423 // The list of manual compactions. The next manual compaction to perform 424 // is at the start of the list. New entries are added to the end. 425 manual []*manualCompaction 426 // inProgress is the set of in-progress flushes and compactions. 427 // It's used in the calculation of some metrics and to initialize L0 428 // sublevels' state. Some of the compactions contained within this 429 // map may have already committed an edit to the version but are 430 // lingering performing cleanup, like deleting obsolete files. 431 inProgress map[*compaction]struct{} 432 433 // rescheduleReadCompaction indicates to an iterator that a read compaction 434 // should be scheduled. 435 rescheduleReadCompaction bool 436 437 // readCompactions is a readCompactionQueue which keeps track of the 438 // compactions which we might have to perform. 439 readCompactions readCompactionQueue 440 441 // The cumulative duration of all completed compactions since Open. 442 // Does not include flushes. 443 duration time.Duration 444 // Flush throughput metric. 445 flushWriteThroughput ThroughputMetric 446 // The idle start time for the flush "loop", i.e., when the flushing 447 // bool above transitions to false. 448 noOngoingFlushStartTime time.Time 449 } 450 451 // Non-zero when file cleaning is disabled. The disabled count acts as a 452 // reference count to prohibit file cleaning. See 453 // DB.{disable,Enable}FileDeletions(). 454 disableFileDeletions int 455 456 snapshots struct { 457 // The list of active snapshots. 458 snapshotList 459 460 // The cumulative count and size of snapshot-pinned keys written to 461 // sstables. 462 cumulativePinnedCount uint64 463 cumulativePinnedSize uint64 464 } 465 466 tableStats struct { 467 // Condition variable used to signal the completion of a 468 // job to collect table stats. 469 cond sync.Cond 470 // True when a stat collection operation is in progress. 471 loading bool 472 // True if stat collection has loaded statistics for all tables 473 // other than those listed explicitly in pending. This flag starts 474 // as false when a database is opened and flips to true once stat 475 // collection has caught up. 476 loadedInitial bool 477 // A slice of files for which stats have not been computed. 478 // Compactions, ingests, flushes append files to be processed. An 479 // active stat collection goroutine clears the list and processes 480 // them. 481 pending []manifest.NewFileEntry 482 } 483 484 tableValidation struct { 485 // cond is a condition variable used to signal the completion of a 486 // job to validate one or more sstables. 487 cond sync.Cond 488 // pending is a slice of metadata for sstables waiting to be 489 // validated. Only physical sstables should be added to the pending 490 // queue. 491 pending []newFileEntry 492 // validating is set to true when validation is running. 493 validating bool 494 } 495 } 496 497 // Normally equal to time.Now() but may be overridden in tests. 498 timeNow func() time.Time 499 // the time at database Open; may be used to compute metrics like effective 500 // compaction concurrency 501 openedAt time.Time 502 } 503 504 var _ Reader = (*DB)(nil) 505 var _ Writer = (*DB)(nil) 506 507 // TestOnlyWaitForCleaning MUST only be used in tests. 508 func (d *DB) TestOnlyWaitForCleaning() { 509 d.cleanupManager.Wait() 510 } 511 512 // Get gets the value for the given key. It returns ErrNotFound if the DB does 513 // not contain the key. 514 // 515 // The caller should not modify the contents of the returned slice, but it is 516 // safe to modify the contents of the argument after Get returns. The returned 517 // slice will remain valid until the returned Closer is closed. On success, the 518 // caller MUST call closer.Close() or a memory leak will occur. 519 func (d *DB) Get(key []byte) ([]byte, io.Closer, error) { 520 return d.getInternal(key, nil /* batch */, nil /* snapshot */) 521 } 522 523 type getIterAlloc struct { 524 dbi Iterator 525 keyBuf []byte 526 get getIter 527 } 528 529 var getIterAllocPool = sync.Pool{ 530 New: func() interface{} { 531 return &getIterAlloc{} 532 }, 533 } 534 535 func (d *DB) getInternal(key []byte, b *Batch, s *Snapshot) ([]byte, io.Closer, error) { 536 if err := d.closed.Load(); err != nil { 537 panic(err) 538 } 539 540 // Grab and reference the current readState. This prevents the underlying 541 // files in the associated version from being deleted if there is a current 542 // compaction. The readState is unref'd by Iterator.Close(). 543 readState := d.loadReadState() 544 545 // Determine the seqnum to read at after grabbing the read state (current and 546 // memtables) above. 547 var seqNum uint64 548 if s != nil { 549 seqNum = s.seqNum 550 } else { 551 seqNum = d.mu.versions.visibleSeqNum.Load() 552 } 553 554 buf := getIterAllocPool.Get().(*getIterAlloc) 555 556 get := &buf.get 557 *get = getIter{ 558 logger: d.opts.Logger, 559 comparer: d.opts.Comparer, 560 newIters: d.newIters, 561 snapshot: seqNum, 562 key: key, 563 batch: b, 564 mem: readState.memtables, 565 l0: readState.current.L0SublevelFiles, 566 version: readState.current, 567 } 568 569 // Strip off memtables which cannot possibly contain the seqNum being read 570 // at. 571 for len(get.mem) > 0 { 572 n := len(get.mem) 573 if logSeqNum := get.mem[n-1].logSeqNum; logSeqNum < seqNum { 574 break 575 } 576 get.mem = get.mem[:n-1] 577 } 578 579 i := &buf.dbi 580 pointIter := get 581 *i = Iterator{ 582 ctx: context.Background(), 583 getIterAlloc: buf, 584 iter: pointIter, 585 pointIter: pointIter, 586 merge: d.merge, 587 comparer: *d.opts.Comparer, 588 readState: readState, 589 keyBuf: buf.keyBuf, 590 } 591 592 if !i.First() { 593 err := i.Close() 594 if err != nil { 595 return nil, nil, err 596 } 597 return nil, nil, ErrNotFound 598 } 599 return i.Value(), i, nil 600 } 601 602 // Set sets the value for the given key. It overwrites any previous value 603 // for that key; a DB is not a multi-map. 604 // 605 // It is safe to modify the contents of the arguments after Set returns. 606 func (d *DB) Set(key, value []byte, opts *WriteOptions) error { 607 b := newBatch(d) 608 _ = b.Set(key, value, opts) 609 if err := d.Apply(b, opts); err != nil { 610 return err 611 } 612 // Only release the batch on success. 613 b.release() 614 return nil 615 } 616 617 // Delete deletes the value for the given key. Deletes are blind all will 618 // succeed even if the given key does not exist. 619 // 620 // It is safe to modify the contents of the arguments after Delete returns. 621 func (d *DB) Delete(key []byte, opts *WriteOptions) error { 622 b := newBatch(d) 623 _ = b.Delete(key, opts) 624 if err := d.Apply(b, opts); err != nil { 625 return err 626 } 627 // Only release the batch on success. 628 b.release() 629 return nil 630 } 631 632 // DeleteSized behaves identically to Delete, but takes an additional 633 // argument indicating the size of the value being deleted. DeleteSized 634 // should be preferred when the caller has the expectation that there exists 635 // a single internal KV pair for the key (eg, the key has not been 636 // overwritten recently), and the caller knows the size of its value. 637 // 638 // DeleteSized will record the value size within the tombstone and use it to 639 // inform compaction-picking heuristics which strive to reduce space 640 // amplification in the LSM. This "calling your shot" mechanic allows the 641 // storage engine to more accurately estimate and reduce space amplification. 642 // 643 // It is safe to modify the contents of the arguments after DeleteSized 644 // returns. 645 func (d *DB) DeleteSized(key []byte, valueSize uint32, opts *WriteOptions) error { 646 b := newBatch(d) 647 _ = b.DeleteSized(key, valueSize, 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 // SingleDelete adds an action to the batch that single deletes the entry for key. 657 // See Writer.SingleDelete for more details on the semantics of SingleDelete. 658 // 659 // It is safe to modify the contents of the arguments after SingleDelete returns. 660 func (d *DB) SingleDelete(key []byte, opts *WriteOptions) error { 661 b := newBatch(d) 662 _ = b.SingleDelete(key, opts) 663 if err := d.Apply(b, opts); err != nil { 664 return err 665 } 666 // Only release the batch on success. 667 b.release() 668 return nil 669 } 670 671 // DeleteRange deletes all of the keys (and values) in the range [start,end) 672 // (inclusive on start, exclusive on end). 673 // 674 // It is safe to modify the contents of the arguments after DeleteRange 675 // returns. 676 func (d *DB) DeleteRange(start, end []byte, opts *WriteOptions) error { 677 b := newBatch(d) 678 _ = b.DeleteRange(start, end, opts) 679 if err := d.Apply(b, opts); err != nil { 680 return err 681 } 682 // Only release the batch on success. 683 b.release() 684 return nil 685 } 686 687 // Merge adds an action to the DB that merges the value at key with the new 688 // value. The details of the merge are dependent upon the configured merge 689 // operator. 690 // 691 // It is safe to modify the contents of the arguments after Merge returns. 692 func (d *DB) Merge(key, value []byte, opts *WriteOptions) error { 693 b := newBatch(d) 694 _ = b.Merge(key, value, opts) 695 if err := d.Apply(b, opts); err != nil { 696 return err 697 } 698 // Only release the batch on success. 699 b.release() 700 return nil 701 } 702 703 // LogData adds the specified to the batch. The data will be written to the 704 // WAL, but not added to memtables or sstables. Log data is never indexed, 705 // which makes it useful for testing WAL performance. 706 // 707 // It is safe to modify the contents of the argument after LogData returns. 708 func (d *DB) LogData(data []byte, opts *WriteOptions) error { 709 b := newBatch(d) 710 _ = b.LogData(data, opts) 711 if err := d.Apply(b, opts); err != nil { 712 return err 713 } 714 // Only release the batch on success. 715 b.release() 716 return nil 717 } 718 719 // RangeKeySet sets a range key mapping the key range [start, end) at the MVCC 720 // timestamp suffix to value. The suffix is optional. If any portion of the key 721 // range [start, end) is already set by a range key with the same suffix value, 722 // RangeKeySet overrides it. 723 // 724 // It is safe to modify the contents of the arguments after RangeKeySet returns. 725 func (d *DB) RangeKeySet(start, end, suffix, value []byte, opts *WriteOptions) error { 726 b := newBatch(d) 727 _ = b.RangeKeySet(start, end, suffix, value, opts) 728 if err := d.Apply(b, opts); err != nil { 729 return err 730 } 731 // Only release the batch on success. 732 b.release() 733 return nil 734 } 735 736 // RangeKeyUnset removes a range key mapping the key range [start, end) at the 737 // MVCC timestamp suffix. The suffix may be omitted to remove an unsuffixed 738 // range key. RangeKeyUnset only removes portions of range keys that fall within 739 // the [start, end) key span, and only range keys with suffixes that exactly 740 // match the unset suffix. 741 // 742 // It is safe to modify the contents of the arguments after RangeKeyUnset 743 // returns. 744 func (d *DB) RangeKeyUnset(start, end, suffix []byte, opts *WriteOptions) error { 745 b := newBatch(d) 746 _ = b.RangeKeyUnset(start, end, suffix, opts) 747 if err := d.Apply(b, opts); err != nil { 748 return err 749 } 750 // Only release the batch on success. 751 b.release() 752 return nil 753 } 754 755 // RangeKeyDelete deletes all of the range keys in the range [start,end) 756 // (inclusive on start, exclusive on end). It does not delete point keys (for 757 // that use DeleteRange). RangeKeyDelete removes all range keys within the 758 // bounds, including those with or without suffixes. 759 // 760 // It is safe to modify the contents of the arguments after RangeKeyDelete 761 // returns. 762 func (d *DB) RangeKeyDelete(start, end []byte, opts *WriteOptions) error { 763 b := newBatch(d) 764 _ = b.RangeKeyDelete(start, end, opts) 765 if err := d.Apply(b, opts); err != nil { 766 return err 767 } 768 // Only release the batch on success. 769 b.release() 770 return nil 771 } 772 773 // Apply the operations contained in the batch to the DB. If the batch is large 774 // the contents of the batch may be retained by the database. If that occurs 775 // the batch contents will be cleared preventing the caller from attempting to 776 // reuse them. 777 // 778 // It is safe to modify the contents of the arguments after Apply returns. 779 func (d *DB) Apply(batch *Batch, opts *WriteOptions) error { 780 return d.applyInternal(batch, opts, false) 781 } 782 783 // ApplyNoSyncWait must only be used when opts.Sync is true and the caller 784 // does not want to wait for the WAL fsync to happen. The method will return 785 // once the mutation is applied to the memtable and is visible (note that a 786 // mutation is visible before the WAL sync even in the wait case, so we have 787 // not weakened the durability semantics). The caller must call Batch.SyncWait 788 // to wait for the WAL fsync. The caller must not Close the batch without 789 // first calling Batch.SyncWait. 790 // 791 // RECOMMENDATION: Prefer using Apply unless you really understand why you 792 // need ApplyNoSyncWait. 793 // EXPERIMENTAL: API/feature subject to change. Do not yet use outside 794 // CockroachDB. 795 func (d *DB) ApplyNoSyncWait(batch *Batch, opts *WriteOptions) error { 796 if !opts.Sync { 797 return errors.Errorf("cannot request asynchonous apply when WriteOptions.Sync is false") 798 } 799 return d.applyInternal(batch, opts, true) 800 } 801 802 // REQUIRES: noSyncWait => opts.Sync 803 func (d *DB) applyInternal(batch *Batch, opts *WriteOptions, noSyncWait bool) error { 804 if err := d.closed.Load(); err != nil { 805 panic(err) 806 } 807 if batch.committing { 808 panic("pebble: batch already committing") 809 } 810 if batch.applied.Load() { 811 panic("pebble: batch already applied") 812 } 813 if d.opts.ReadOnly { 814 return ErrReadOnly 815 } 816 if batch.db != nil && batch.db != d { 817 panic(fmt.Sprintf("pebble: batch db mismatch: %p != %p", batch.db, d)) 818 } 819 820 sync := opts.GetSync() 821 if sync && d.opts.DisableWAL { 822 return errors.New("pebble: WAL disabled") 823 } 824 825 if batch.minimumFormatMajorVersion != FormatMostCompatible { 826 if fmv := d.FormatMajorVersion(); fmv < batch.minimumFormatMajorVersion { 827 panic(fmt.Sprintf( 828 "pebble: batch requires at least format major version %d (current: %d)", 829 batch.minimumFormatMajorVersion, fmv, 830 )) 831 } 832 } 833 834 if batch.countRangeKeys > 0 { 835 if d.split == nil { 836 return errNoSplit 837 } 838 // TODO(jackson): Assert that all range key operands are suffixless. 839 } 840 batch.committing = true 841 842 if batch.db == nil { 843 if err := batch.refreshMemTableSize(); err != nil { 844 return err 845 } 846 } 847 if batch.memTableSize >= d.largeBatchThreshold { 848 var err error 849 batch.flushable, err = newFlushableBatch(batch, d.opts.Comparer) 850 if err != nil { 851 return err 852 } 853 } 854 if err := d.commit.Commit(batch, sync, noSyncWait); err != nil { 855 // There isn't much we can do on an error here. The commit pipeline will be 856 // horked at this point. 857 d.opts.Logger.Fatalf("pebble: fatal commit error: %v", err) 858 } 859 // If this is a large batch, we need to clear the batch contents as the 860 // flushable batch may still be present in the flushables queue. 861 // 862 // TODO(peter): Currently large batches are written to the WAL. We could 863 // skip the WAL write and instead wait for the large batch to be flushed to 864 // an sstable. For a 100 MB batch, this might actually be faster. For a 1 865 // GB batch this is almost certainly faster. 866 if batch.flushable != nil { 867 batch.data = nil 868 } 869 return nil 870 } 871 872 func (d *DB) commitApply(b *Batch, mem *memTable) error { 873 if b.flushable != nil { 874 // This is a large batch which was already added to the immutable queue. 875 return nil 876 } 877 err := mem.apply(b, b.SeqNum()) 878 if err != nil { 879 return err 880 } 881 882 // If the batch contains range tombstones and the database is configured 883 // to flush range deletions, schedule a delayed flush so that disk space 884 // may be reclaimed without additional writes or an explicit flush. 885 if b.countRangeDels > 0 && d.opts.FlushDelayDeleteRange > 0 { 886 d.mu.Lock() 887 d.maybeScheduleDelayedFlush(mem, d.opts.FlushDelayDeleteRange) 888 d.mu.Unlock() 889 } 890 891 // If the batch contains range keys and the database is configured to flush 892 // range keys, schedule a delayed flush so that the range keys are cleared 893 // from the memtable. 894 if b.countRangeKeys > 0 && d.opts.FlushDelayRangeKey > 0 { 895 d.mu.Lock() 896 d.maybeScheduleDelayedFlush(mem, d.opts.FlushDelayRangeKey) 897 d.mu.Unlock() 898 } 899 900 if mem.writerUnref() { 901 d.mu.Lock() 902 d.maybeScheduleFlush() 903 d.mu.Unlock() 904 } 905 return nil 906 } 907 908 func (d *DB) commitWrite(b *Batch, syncWG *sync.WaitGroup, syncErr *error) (*memTable, error) { 909 var size int64 910 repr := b.Repr() 911 912 if b.flushable != nil { 913 // We have a large batch. Such batches are special in that they don't get 914 // added to the memtable, and are instead inserted into the queue of 915 // memtables. The call to makeRoomForWrite with this batch will force the 916 // current memtable to be flushed. We want the large batch to be part of 917 // the same log, so we add it to the WAL here, rather than after the call 918 // to makeRoomForWrite(). 919 // 920 // Set the sequence number since it was not set to the correct value earlier 921 // (see comment in newFlushableBatch()). 922 b.flushable.setSeqNum(b.SeqNum()) 923 if !d.opts.DisableWAL { 924 var err error 925 size, err = d.mu.log.SyncRecord(repr, syncWG, syncErr) 926 if err != nil { 927 panic(err) 928 } 929 } 930 } 931 932 d.mu.Lock() 933 934 var err error 935 if !b.ingestedSSTBatch { 936 // Batches which contain keys of kind InternalKeyKindIngestSST will 937 // never be applied to the memtable, so we don't need to make room for 938 // write. For the other cases, switch out the memtable if there was not 939 // enough room to store the batch. 940 err = d.makeRoomForWrite(b) 941 } 942 943 if err == nil && !d.opts.DisableWAL { 944 d.mu.log.bytesIn += uint64(len(repr)) 945 } 946 947 // Grab a reference to the memtable while holding DB.mu. Note that for 948 // non-flushable batches (b.flushable == nil) makeRoomForWrite() added a 949 // reference to the memtable which will prevent it from being flushed until 950 // we unreference it. This reference is dropped in DB.commitApply(). 951 mem := d.mu.mem.mutable 952 953 d.mu.Unlock() 954 if err != nil { 955 return nil, err 956 } 957 958 if d.opts.DisableWAL { 959 return mem, nil 960 } 961 962 if b.flushable == nil { 963 size, err = d.mu.log.SyncRecord(repr, syncWG, syncErr) 964 if err != nil { 965 panic(err) 966 } 967 } 968 969 d.logSize.Store(uint64(size)) 970 return mem, err 971 } 972 973 type iterAlloc struct { 974 dbi Iterator 975 keyBuf []byte 976 boundsBuf [2][]byte 977 prefixOrFullSeekKey []byte 978 merging mergingIter 979 mlevels [3 + numLevels]mergingIterLevel 980 levels [3 + numLevels]levelIter 981 levelsPositioned [3 + numLevels]bool 982 } 983 984 var iterAllocPool = sync.Pool{ 985 New: func() interface{} { 986 return &iterAlloc{} 987 }, 988 } 989 990 // snapshotIterOpts denotes snapshot-related iterator options when calling 991 // newIter. These are the possible cases for a snapshotIterOpts: 992 // - No snapshot: All fields are zero values. 993 // - Classic snapshot: Only `seqNum` is set. The latest readState will be used 994 // and the specified seqNum will be used as the snapshot seqNum. 995 // - EventuallyFileOnlySnapshot (EFOS) behaving as a classic snapshot. Only 996 // the `seqNum` is set. The latest readState will be used 997 // and the specified seqNum will be used as the snapshot seqNum. 998 // - EFOS in file-only state: Only `seqNum` and `vers` are set. All the 999 // relevant SSTs are referenced by the *version. 1000 type snapshotIterOpts struct { 1001 seqNum uint64 1002 vers *version 1003 } 1004 1005 // newIter constructs a new iterator, merging in batch iterators as an extra 1006 // level. 1007 func (d *DB) newIter( 1008 ctx context.Context, batch *Batch, sOpts snapshotIterOpts, o *IterOptions, 1009 ) *Iterator { 1010 if err := d.closed.Load(); err != nil { 1011 panic(err) 1012 } 1013 seqNum := sOpts.seqNum 1014 if o.rangeKeys() { 1015 if d.FormatMajorVersion() < FormatRangeKeys { 1016 panic(fmt.Sprintf( 1017 "pebble: range keys require at least format major version %d (current: %d)", 1018 FormatRangeKeys, d.FormatMajorVersion(), 1019 )) 1020 } 1021 } 1022 if o != nil && o.RangeKeyMasking.Suffix != nil && o.KeyTypes != IterKeyTypePointsAndRanges { 1023 panic("pebble: range key masking requires IterKeyTypePointsAndRanges") 1024 } 1025 if (batch != nil || seqNum != 0) && (o != nil && o.OnlyReadGuaranteedDurable) { 1026 // We could add support for OnlyReadGuaranteedDurable on snapshots if 1027 // there was a need: this would require checking that the sequence number 1028 // of the snapshot has been flushed, by comparing with 1029 // DB.mem.queue[0].logSeqNum. 1030 panic("OnlyReadGuaranteedDurable is not supported for batches or snapshots") 1031 } 1032 // Grab and reference the current readState. This prevents the underlying 1033 // files in the associated version from being deleted if there is a current 1034 // compaction. The readState is unref'd by Iterator.Close(). 1035 var readState *readState 1036 if sOpts.vers == nil { 1037 // NB: loadReadState() calls readState.ref(). 1038 readState = d.loadReadState() 1039 } else { 1040 // s.vers != nil 1041 sOpts.vers.Ref() 1042 } 1043 1044 // Determine the seqnum to read at after grabbing the read state (current and 1045 // memtables) above. 1046 if seqNum == 0 { 1047 seqNum = d.mu.versions.visibleSeqNum.Load() 1048 } 1049 1050 // Bundle various structures under a single umbrella in order to allocate 1051 // them together. 1052 buf := iterAllocPool.Get().(*iterAlloc) 1053 dbi := &buf.dbi 1054 *dbi = Iterator{ 1055 ctx: ctx, 1056 alloc: buf, 1057 merge: d.merge, 1058 comparer: *d.opts.Comparer, 1059 readState: readState, 1060 version: sOpts.vers, 1061 keyBuf: buf.keyBuf, 1062 prefixOrFullSeekKey: buf.prefixOrFullSeekKey, 1063 boundsBuf: buf.boundsBuf, 1064 batch: batch, 1065 newIters: d.newIters, 1066 newIterRangeKey: d.tableNewRangeKeyIter, 1067 seqNum: seqNum, 1068 } 1069 if o != nil { 1070 dbi.opts = *o 1071 dbi.processBounds(o.LowerBound, o.UpperBound) 1072 } 1073 dbi.opts.logger = d.opts.Logger 1074 if d.opts.private.disableLazyCombinedIteration { 1075 dbi.opts.disableLazyCombinedIteration = true 1076 } 1077 if batch != nil { 1078 dbi.batchSeqNum = dbi.batch.nextSeqNum() 1079 } 1080 return finishInitializingIter(ctx, buf) 1081 } 1082 1083 // finishInitializingIter is a helper for doing the non-trivial initialization 1084 // of an Iterator. It's invoked to perform the initial initialization of an 1085 // Iterator during NewIter or Clone, and to perform reinitialization due to a 1086 // change in IterOptions by a call to Iterator.SetOptions. 1087 func finishInitializingIter(ctx context.Context, buf *iterAlloc) *Iterator { 1088 // Short-hand. 1089 dbi := &buf.dbi 1090 var memtables flushableList 1091 if dbi.readState != nil { 1092 memtables = dbi.readState.memtables 1093 } 1094 if dbi.opts.OnlyReadGuaranteedDurable { 1095 memtables = nil 1096 } else { 1097 // We only need to read from memtables which contain sequence numbers older 1098 // than seqNum. Trim off newer memtables. 1099 for i := len(memtables) - 1; i >= 0; i-- { 1100 if logSeqNum := memtables[i].logSeqNum; logSeqNum < dbi.seqNum { 1101 break 1102 } 1103 memtables = memtables[:i] 1104 } 1105 } 1106 1107 if dbi.opts.pointKeys() { 1108 // Construct the point iterator, initializing dbi.pointIter to point to 1109 // dbi.merging. If this is called during a SetOptions call and this 1110 // Iterator has already initialized dbi.merging, constructPointIter is a 1111 // noop and an initialized pointIter already exists in dbi.pointIter. 1112 dbi.constructPointIter(ctx, memtables, buf) 1113 dbi.iter = dbi.pointIter 1114 } else { 1115 dbi.iter = emptyIter 1116 } 1117 1118 if dbi.opts.rangeKeys() { 1119 dbi.rangeKeyMasking.init(dbi, dbi.comparer.Compare, dbi.comparer.Split) 1120 1121 // When iterating over both point and range keys, don't create the 1122 // range-key iterator stack immediately if we can avoid it. This 1123 // optimization takes advantage of the expected sparseness of range 1124 // keys, and configures the point-key iterator to dynamically switch to 1125 // combined iteration when it observes a file containing range keys. 1126 // 1127 // Lazy combined iteration is not possible if a batch or a memtable 1128 // contains any range keys. 1129 useLazyCombinedIteration := dbi.rangeKey == nil && 1130 dbi.opts.KeyTypes == IterKeyTypePointsAndRanges && 1131 (dbi.batch == nil || dbi.batch.countRangeKeys == 0) && 1132 !dbi.opts.disableLazyCombinedIteration 1133 if useLazyCombinedIteration { 1134 // The user requested combined iteration, and there's no indexed 1135 // batch currently containing range keys that would prevent lazy 1136 // combined iteration. Check the memtables to see if they contain 1137 // any range keys. 1138 for i := range memtables { 1139 if memtables[i].containsRangeKeys() { 1140 useLazyCombinedIteration = false 1141 break 1142 } 1143 } 1144 } 1145 1146 if useLazyCombinedIteration { 1147 dbi.lazyCombinedIter = lazyCombinedIter{ 1148 parent: dbi, 1149 pointIter: dbi.pointIter, 1150 combinedIterState: combinedIterState{ 1151 initialized: false, 1152 }, 1153 } 1154 dbi.iter = &dbi.lazyCombinedIter 1155 dbi.iter = invalidating.MaybeWrapIfInvariants(dbi.iter) 1156 } else { 1157 dbi.lazyCombinedIter.combinedIterState = combinedIterState{ 1158 initialized: true, 1159 } 1160 if dbi.rangeKey == nil { 1161 dbi.rangeKey = iterRangeKeyStateAllocPool.Get().(*iteratorRangeKeyState) 1162 dbi.rangeKey.init(dbi.comparer.Compare, dbi.comparer.Split, &dbi.opts) 1163 dbi.constructRangeKeyIter() 1164 } else { 1165 dbi.rangeKey.iterConfig.SetBounds(dbi.opts.LowerBound, dbi.opts.UpperBound) 1166 } 1167 1168 // Wrap the point iterator (currently dbi.iter) with an interleaving 1169 // iterator that interleaves range keys pulled from 1170 // dbi.rangeKey.rangeKeyIter. 1171 // 1172 // NB: The interleaving iterator is always reinitialized, even if 1173 // dbi already had an initialized range key iterator, in case the point 1174 // iterator changed or the range key masking suffix changed. 1175 dbi.rangeKey.iiter.Init(&dbi.comparer, dbi.iter, dbi.rangeKey.rangeKeyIter, 1176 keyspan.InterleavingIterOpts{ 1177 Mask: &dbi.rangeKeyMasking, 1178 LowerBound: dbi.opts.LowerBound, 1179 UpperBound: dbi.opts.UpperBound, 1180 }) 1181 dbi.iter = &dbi.rangeKey.iiter 1182 } 1183 } else { 1184 // !dbi.opts.rangeKeys() 1185 // 1186 // Reset the combined iterator state. The initialized=true ensures the 1187 // iterator doesn't unnecessarily try to switch to combined iteration. 1188 dbi.lazyCombinedIter.combinedIterState = combinedIterState{initialized: true} 1189 } 1190 return dbi 1191 } 1192 1193 // ScanInternal scans all internal keys within the specified bounds, truncating 1194 // any rangedels and rangekeys to those bounds if they span past them. For use 1195 // when an external user needs to be aware of all internal keys that make up a 1196 // key range. 1197 // 1198 // Keys deleted by range deletions must not be returned or exposed by this 1199 // method, while the range deletion deleting that key must be exposed using 1200 // visitRangeDel. Keys that would be masked by range key masking (if an 1201 // appropriate prefix were set) should be exposed, alongside the range key 1202 // that would have masked it. This method also collapses all point keys into 1203 // one InternalKey; so only one internal key at most per user key is returned 1204 // to visitPointKey. 1205 // 1206 // If visitSharedFile is not nil, ScanInternal iterates in skip-shared iteration 1207 // mode. In this iteration mode, sstables in levels L5 and L6 are skipped, and 1208 // their metadatas truncated to [lower, upper) and passed into visitSharedFile. 1209 // ErrInvalidSkipSharedIteration is returned if visitSharedFile is not nil and an 1210 // sstable in L5 or L6 is found that is not in shared storage according to 1211 // provider.IsShared, or an sstable in those levels contains a newer key than the 1212 // snapshot sequence number (only applicable for snapshot.ScanInternal). Examples 1213 // of when this could happen could be if Pebble started writing sstables before a 1214 // creator ID was set (as creator IDs are necessary to enable shared storage) 1215 // resulting in some lower level SSTs being on non-shared storage. Skip-shared 1216 // iteration is invalid in those cases. 1217 func (d *DB) ScanInternal( 1218 ctx context.Context, 1219 lower, upper []byte, 1220 visitPointKey func(key *InternalKey, value LazyValue, iterInfo IteratorLevel) error, 1221 visitRangeDel func(start, end []byte, seqNum uint64) error, 1222 visitRangeKey func(start, end []byte, keys []rangekey.Key) error, 1223 visitSharedFile func(sst *SharedSSTMeta) error, 1224 ) error { 1225 scanInternalOpts := &scanInternalOptions{ 1226 visitPointKey: visitPointKey, 1227 visitRangeDel: visitRangeDel, 1228 visitRangeKey: visitRangeKey, 1229 visitSharedFile: visitSharedFile, 1230 skipSharedLevels: visitSharedFile != nil, 1231 IterOptions: IterOptions{ 1232 KeyTypes: IterKeyTypePointsAndRanges, 1233 LowerBound: lower, 1234 UpperBound: upper, 1235 }, 1236 } 1237 iter := d.newInternalIter(snapshotIterOpts{} /* snapshot */, scanInternalOpts) 1238 defer iter.close() 1239 return scanInternalImpl(ctx, lower, upper, iter, scanInternalOpts) 1240 } 1241 1242 // newInternalIter constructs and returns a new scanInternalIterator on this db. 1243 // If o.skipSharedLevels is true, levels below sharedLevelsStart are *not* added 1244 // to the internal iterator. 1245 // 1246 // TODO(bilal): This method has a lot of similarities with db.newIter as well as 1247 // finishInitializingIter. Both pairs of methods should be refactored to reduce 1248 // this duplication. 1249 func (d *DB) newInternalIter(sOpts snapshotIterOpts, o *scanInternalOptions) *scanInternalIterator { 1250 if err := d.closed.Load(); err != nil { 1251 panic(err) 1252 } 1253 // Grab and reference the current readState. This prevents the underlying 1254 // files in the associated version from being deleted if there is a current 1255 // compaction. The readState is unref'd by Iterator.Close(). 1256 var readState *readState 1257 if sOpts.vers == nil { 1258 readState = d.loadReadState() 1259 } 1260 if sOpts.vers != nil { 1261 sOpts.vers.Ref() 1262 } 1263 1264 // Determine the seqnum to read at after grabbing the read state (current and 1265 // memtables) above. 1266 seqNum := sOpts.seqNum 1267 if seqNum == 0 { 1268 seqNum = d.mu.versions.visibleSeqNum.Load() 1269 } 1270 1271 // Bundle various structures under a single umbrella in order to allocate 1272 // them together. 1273 buf := iterAllocPool.Get().(*iterAlloc) 1274 dbi := &scanInternalIterator{ 1275 db: d, 1276 comparer: d.opts.Comparer, 1277 merge: d.opts.Merger.Merge, 1278 readState: readState, 1279 version: sOpts.vers, 1280 alloc: buf, 1281 newIters: d.newIters, 1282 newIterRangeKey: d.tableNewRangeKeyIter, 1283 seqNum: seqNum, 1284 mergingIter: &buf.merging, 1285 } 1286 if o != nil { 1287 dbi.opts = *o 1288 } 1289 dbi.opts.logger = d.opts.Logger 1290 if d.opts.private.disableLazyCombinedIteration { 1291 dbi.opts.disableLazyCombinedIteration = true 1292 } 1293 return finishInitializingInternalIter(buf, dbi) 1294 } 1295 1296 func finishInitializingInternalIter(buf *iterAlloc, i *scanInternalIterator) *scanInternalIterator { 1297 // Short-hand. 1298 var memtables flushableList 1299 if i.readState != nil { 1300 memtables = i.readState.memtables 1301 } 1302 // We only need to read from memtables which contain sequence numbers older 1303 // than seqNum. Trim off newer memtables. 1304 for j := len(memtables) - 1; j >= 0; j-- { 1305 if logSeqNum := memtables[j].logSeqNum; logSeqNum < i.seqNum { 1306 break 1307 } 1308 memtables = memtables[:j] 1309 } 1310 i.initializeBoundBufs(i.opts.LowerBound, i.opts.UpperBound) 1311 1312 i.constructPointIter(memtables, buf) 1313 1314 // For internal iterators, we skip the lazy combined iteration optimization 1315 // entirely, and create the range key iterator stack directly. 1316 i.rangeKey = iterRangeKeyStateAllocPool.Get().(*iteratorRangeKeyState) 1317 i.rangeKey.init(i.comparer.Compare, i.comparer.Split, &i.opts.IterOptions) 1318 i.constructRangeKeyIter() 1319 1320 // Wrap the point iterator (currently i.iter) with an interleaving 1321 // iterator that interleaves range keys pulled from 1322 // i.rangeKey.rangeKeyIter. 1323 i.rangeKey.iiter.Init(i.comparer, i.iter, i.rangeKey.rangeKeyIter, 1324 keyspan.InterleavingIterOpts{ 1325 LowerBound: i.opts.LowerBound, 1326 UpperBound: i.opts.UpperBound, 1327 }) 1328 i.iter = &i.rangeKey.iiter 1329 1330 return i 1331 } 1332 1333 func (i *Iterator) constructPointIter( 1334 ctx context.Context, memtables flushableList, buf *iterAlloc, 1335 ) { 1336 if i.pointIter != nil { 1337 // Already have one. 1338 return 1339 } 1340 internalOpts := internalIterOpts{stats: &i.stats.InternalStats} 1341 if i.opts.RangeKeyMasking.Filter != nil { 1342 internalOpts.boundLimitedFilter = &i.rangeKeyMasking 1343 } 1344 1345 // Merging levels and levels from iterAlloc. 1346 mlevels := buf.mlevels[:0] 1347 levels := buf.levels[:0] 1348 1349 // We compute the number of levels needed ahead of time and reallocate a slice if 1350 // the array from the iterAlloc isn't large enough. Doing this allocation once 1351 // should improve the performance. 1352 numMergingLevels := 0 1353 numLevelIters := 0 1354 if i.batch != nil { 1355 numMergingLevels++ 1356 } 1357 numMergingLevels += len(memtables) 1358 1359 current := i.version 1360 if current == nil { 1361 current = i.readState.current 1362 } 1363 numMergingLevels += len(current.L0SublevelFiles) 1364 numLevelIters += len(current.L0SublevelFiles) 1365 for level := 1; level < len(current.Levels); level++ { 1366 if current.Levels[level].Empty() { 1367 continue 1368 } 1369 numMergingLevels++ 1370 numLevelIters++ 1371 } 1372 1373 if numMergingLevels > cap(mlevels) { 1374 mlevels = make([]mergingIterLevel, 0, numMergingLevels) 1375 } 1376 if numLevelIters > cap(levels) { 1377 levels = make([]levelIter, 0, numLevelIters) 1378 } 1379 1380 // Top-level is the batch, if any. 1381 if i.batch != nil { 1382 if i.batch.index == nil { 1383 // This isn't an indexed batch. Include an error iterator so that 1384 // the resulting iterator correctly surfaces ErrIndexed. 1385 mlevels = append(mlevels, mergingIterLevel{ 1386 iter: newErrorIter(ErrNotIndexed), 1387 rangeDelIter: newErrorKeyspanIter(ErrNotIndexed), 1388 }) 1389 } else { 1390 i.batch.initInternalIter(&i.opts, &i.batchPointIter) 1391 i.batch.initRangeDelIter(&i.opts, &i.batchRangeDelIter, i.batchSeqNum) 1392 // Only include the batch's rangedel iterator if it's non-empty. 1393 // This requires some subtle logic in the case a rangedel is later 1394 // written to the batch and the view of the batch is refreshed 1395 // during a call to SetOptions—in this case, we need to reconstruct 1396 // the point iterator to add the batch rangedel iterator. 1397 var rangeDelIter keyspan.FragmentIterator 1398 if i.batchRangeDelIter.Count() > 0 { 1399 rangeDelIter = &i.batchRangeDelIter 1400 } 1401 mlevels = append(mlevels, mergingIterLevel{ 1402 iter: &i.batchPointIter, 1403 rangeDelIter: rangeDelIter, 1404 }) 1405 } 1406 } 1407 1408 // Next are the memtables. 1409 for j := len(memtables) - 1; j >= 0; j-- { 1410 mem := memtables[j] 1411 mlevels = append(mlevels, mergingIterLevel{ 1412 iter: mem.newIter(&i.opts), 1413 rangeDelIter: mem.newRangeDelIter(&i.opts), 1414 }) 1415 } 1416 1417 // Next are the file levels: L0 sub-levels followed by lower levels. 1418 mlevelsIndex := len(mlevels) 1419 levelsIndex := len(levels) 1420 mlevels = mlevels[:numMergingLevels] 1421 levels = levels[:numLevelIters] 1422 i.opts.snapshotForHideObsoletePoints = buf.dbi.seqNum 1423 addLevelIterForFiles := func(files manifest.LevelIterator, level manifest.Level) { 1424 li := &levels[levelsIndex] 1425 1426 li.init(ctx, i.opts, &i.comparer, i.newIters, files, level, internalOpts) 1427 li.initRangeDel(&mlevels[mlevelsIndex].rangeDelIter) 1428 li.initBoundaryContext(&mlevels[mlevelsIndex].levelIterBoundaryContext) 1429 li.initCombinedIterState(&i.lazyCombinedIter.combinedIterState) 1430 mlevels[mlevelsIndex].levelIter = li 1431 mlevels[mlevelsIndex].iter = invalidating.MaybeWrapIfInvariants(li) 1432 1433 levelsIndex++ 1434 mlevelsIndex++ 1435 } 1436 1437 // Add level iterators for the L0 sublevels, iterating from newest to 1438 // oldest. 1439 for i := len(current.L0SublevelFiles) - 1; i >= 0; i-- { 1440 addLevelIterForFiles(current.L0SublevelFiles[i].Iter(), manifest.L0Sublevel(i)) 1441 } 1442 1443 // Add level iterators for the non-empty non-L0 levels. 1444 for level := 1; level < len(current.Levels); level++ { 1445 if current.Levels[level].Empty() { 1446 continue 1447 } 1448 addLevelIterForFiles(current.Levels[level].Iter(), manifest.Level(level)) 1449 } 1450 buf.merging.init(&i.opts, &i.stats.InternalStats, i.comparer.Compare, i.comparer.Split, mlevels...) 1451 if len(mlevels) <= cap(buf.levelsPositioned) { 1452 buf.merging.levelsPositioned = buf.levelsPositioned[:len(mlevels)] 1453 } 1454 buf.merging.snapshot = i.seqNum 1455 buf.merging.batchSnapshot = i.batchSeqNum 1456 buf.merging.combinedIterState = &i.lazyCombinedIter.combinedIterState 1457 i.pointIter = invalidating.MaybeWrapIfInvariants(&buf.merging) 1458 i.merging = &buf.merging 1459 } 1460 1461 // NewBatch returns a new empty write-only batch. Any reads on the batch will 1462 // return an error. If the batch is committed it will be applied to the DB. 1463 func (d *DB) NewBatch() *Batch { 1464 return newBatch(d) 1465 } 1466 1467 // NewBatchWithSize is mostly identical to NewBatch, but it will allocate the 1468 // the specified memory space for the internal slice in advance. 1469 func (d *DB) NewBatchWithSize(size int) *Batch { 1470 return newBatchWithSize(d, size) 1471 } 1472 1473 // NewIndexedBatch returns a new empty read-write batch. Any reads on the batch 1474 // will read from both the batch and the DB. If the batch is committed it will 1475 // be applied to the DB. An indexed batch is slower that a non-indexed batch 1476 // for insert operations. If you do not need to perform reads on the batch, use 1477 // NewBatch instead. 1478 func (d *DB) NewIndexedBatch() *Batch { 1479 return newIndexedBatch(d, d.opts.Comparer) 1480 } 1481 1482 // NewIndexedBatchWithSize is mostly identical to NewIndexedBatch, but it will 1483 // allocate the the specified memory space for the internal slice in advance. 1484 func (d *DB) NewIndexedBatchWithSize(size int) *Batch { 1485 return newIndexedBatchWithSize(d, d.opts.Comparer, size) 1486 } 1487 1488 // NewIter returns an iterator that is unpositioned (Iterator.Valid() will 1489 // return false). The iterator can be positioned via a call to SeekGE, SeekLT, 1490 // First or Last. The iterator provides a point-in-time view of the current DB 1491 // state. This view is maintained by preventing file deletions and preventing 1492 // memtables referenced by the iterator from being deleted. Using an iterator 1493 // to maintain a long-lived point-in-time view of the DB state can lead to an 1494 // apparent memory and disk usage leak. Use snapshots (see NewSnapshot) for 1495 // point-in-time snapshots which avoids these problems. 1496 func (d *DB) NewIter(o *IterOptions) (*Iterator, error) { 1497 return d.NewIterWithContext(context.Background(), o) 1498 } 1499 1500 // NewIterWithContext is like NewIter, and additionally accepts a context for 1501 // tracing. 1502 func (d *DB) NewIterWithContext(ctx context.Context, o *IterOptions) (*Iterator, error) { 1503 return d.newIter(ctx, nil /* batch */, snapshotIterOpts{}, o), nil 1504 } 1505 1506 // NewSnapshot returns a point-in-time view of the current DB state. Iterators 1507 // created with this handle will all observe a stable snapshot of the current 1508 // DB state. The caller must call Snapshot.Close() when the snapshot is no 1509 // longer needed. Snapshots are not persisted across DB restarts (close -> 1510 // open). Unlike the implicit snapshot maintained by an iterator, a snapshot 1511 // will not prevent memtables from being released or sstables from being 1512 // deleted. Instead, a snapshot prevents deletion of sequence numbers 1513 // referenced by the snapshot. 1514 func (d *DB) NewSnapshot() *Snapshot { 1515 if err := d.closed.Load(); err != nil { 1516 panic(err) 1517 } 1518 1519 d.mu.Lock() 1520 s := &Snapshot{ 1521 db: d, 1522 seqNum: d.mu.versions.visibleSeqNum.Load(), 1523 } 1524 d.mu.snapshots.pushBack(s) 1525 d.mu.Unlock() 1526 return s 1527 } 1528 1529 // NewEventuallyFileOnlySnapshot returns a point-in-time view of the current DB 1530 // state, similar to NewSnapshot, but with consistency constrained to the 1531 // provided set of key ranges. See the comment at EventuallyFileOnlySnapshot for 1532 // its semantics. 1533 func (d *DB) NewEventuallyFileOnlySnapshot(keyRanges []KeyRange) *EventuallyFileOnlySnapshot { 1534 if err := d.closed.Load(); err != nil { 1535 panic(err) 1536 } 1537 1538 internalKeyRanges := make([]internalKeyRange, len(keyRanges)) 1539 for i := range keyRanges { 1540 if i > 0 && d.cmp(keyRanges[i-1].End, keyRanges[i].Start) > 0 { 1541 panic("pebble: key ranges for eventually-file-only-snapshot not in order") 1542 } 1543 internalKeyRanges[i] = internalKeyRange{ 1544 smallest: base.MakeInternalKey(keyRanges[i].Start, InternalKeySeqNumMax, InternalKeyKindMax), 1545 largest: base.MakeExclusiveSentinelKey(InternalKeyKindRangeDelete, keyRanges[i].End), 1546 } 1547 } 1548 1549 return d.makeEventuallyFileOnlySnapshot(keyRanges, internalKeyRanges) 1550 } 1551 1552 // Close closes the DB. 1553 // 1554 // It is not safe to close a DB until all outstanding iterators are closed 1555 // or to call Close concurrently with any other DB method. It is not valid 1556 // to call any of a DB's methods after the DB has been closed. 1557 func (d *DB) Close() error { 1558 // Lock the commit pipeline for the duration of Close. This prevents a race 1559 // with makeRoomForWrite. Rotating the WAL in makeRoomForWrite requires 1560 // dropping d.mu several times for I/O. If Close only holds d.mu, an 1561 // in-progress WAL rotation may re-acquire d.mu only once the database is 1562 // closed. 1563 // 1564 // Additionally, locking the commit pipeline makes it more likely that 1565 // (illegal) concurrent writes will observe d.closed.Load() != nil, creating 1566 // more understable panics if the database is improperly used concurrently 1567 // during Close. 1568 d.commit.mu.Lock() 1569 defer d.commit.mu.Unlock() 1570 d.mu.Lock() 1571 defer d.mu.Unlock() 1572 if err := d.closed.Load(); err != nil { 1573 panic(err) 1574 } 1575 1576 // Clear the finalizer that is used to check that an unreferenced DB has been 1577 // closed. We're closing the DB here, so the check performed by that 1578 // finalizer isn't necessary. 1579 // 1580 // Note: this is a no-op if invariants are disabled or race is enabled. 1581 invariants.SetFinalizer(d.closed, nil) 1582 1583 d.closed.Store(errors.WithStack(ErrClosed)) 1584 close(d.closedCh) 1585 1586 defer d.opts.Cache.Unref() 1587 1588 for d.mu.compact.compactingCount > 0 || d.mu.compact.flushing { 1589 d.mu.compact.cond.Wait() 1590 } 1591 for d.mu.tableStats.loading { 1592 d.mu.tableStats.cond.Wait() 1593 } 1594 for d.mu.tableValidation.validating { 1595 d.mu.tableValidation.cond.Wait() 1596 } 1597 1598 var err error 1599 if n := len(d.mu.compact.inProgress); n > 0 { 1600 err = errors.Errorf("pebble: %d unexpected in-progress compactions", errors.Safe(n)) 1601 } 1602 err = firstError(err, d.mu.formatVers.marker.Close()) 1603 err = firstError(err, d.tableCache.close()) 1604 if !d.opts.ReadOnly { 1605 err = firstError(err, d.mu.log.Close()) 1606 } else if d.mu.log.LogWriter != nil { 1607 panic("pebble: log-writer should be nil in read-only mode") 1608 } 1609 err = firstError(err, d.fileLock.Close()) 1610 1611 // Note that versionSet.close() only closes the MANIFEST. The versions list 1612 // is still valid for the checks below. 1613 err = firstError(err, d.mu.versions.close()) 1614 1615 err = firstError(err, d.dataDir.Close()) 1616 if d.dataDir != d.walDir { 1617 err = firstError(err, d.walDir.Close()) 1618 } 1619 1620 d.readState.val.unrefLocked() 1621 1622 current := d.mu.versions.currentVersion() 1623 for v := d.mu.versions.versions.Front(); true; v = v.Next() { 1624 refs := v.Refs() 1625 if v == current { 1626 if refs != 1 { 1627 err = firstError(err, errors.Errorf("leaked iterators: current\n%s", v)) 1628 } 1629 break 1630 } 1631 if refs != 0 { 1632 err = firstError(err, errors.Errorf("leaked iterators:\n%s", v)) 1633 } 1634 } 1635 1636 for _, mem := range d.mu.mem.queue { 1637 // Usually, we'd want to delete the files returned by readerUnref. But 1638 // in this case, even if we're unreferencing the flushables, the 1639 // flushables aren't obsolete. They will be reconstructed during WAL 1640 // replay. 1641 mem.readerUnrefLocked(false) 1642 } 1643 // If there's an unused, recycled memtable, we need to release its memory. 1644 if obsoleteMemTable := d.memTableRecycle.Swap(nil); obsoleteMemTable != nil { 1645 d.freeMemTable(obsoleteMemTable) 1646 } 1647 if reserved := d.memTableReserved.Load(); reserved != 0 { 1648 err = firstError(err, errors.Errorf("leaked memtable reservation: %d", errors.Safe(reserved))) 1649 } 1650 1651 // Since we called d.readState.val.unrefLocked() above, we are expected to 1652 // manually schedule deletion of obsolete files. 1653 if len(d.mu.versions.obsoleteTables) > 0 { 1654 d.deleteObsoleteFiles(d.mu.nextJobID) 1655 } 1656 1657 d.mu.Unlock() 1658 d.compactionSchedulers.Wait() 1659 1660 // Wait for all cleaning jobs to finish. 1661 d.cleanupManager.Close() 1662 1663 // Sanity check metrics. 1664 if invariants.Enabled { 1665 m := d.Metrics() 1666 if m.Compact.NumInProgress > 0 || m.Compact.InProgressBytes > 0 { 1667 d.mu.Lock() 1668 panic(fmt.Sprintf("invalid metrics on close:\n%s", m)) 1669 } 1670 } 1671 1672 d.mu.Lock() 1673 1674 // As a sanity check, ensure that there are no zombie tables. A non-zero count 1675 // hints at a reference count leak. 1676 if ztbls := len(d.mu.versions.zombieTables); ztbls > 0 { 1677 err = firstError(err, errors.Errorf("non-zero zombie file count: %d", ztbls)) 1678 } 1679 1680 err = firstError(err, d.objProvider.Close()) 1681 1682 // If the options include a closer to 'close' the filesystem, close it. 1683 if d.opts.private.fsCloser != nil { 1684 d.opts.private.fsCloser.Close() 1685 } 1686 1687 // Return an error if the user failed to close all open snapshots. 1688 if v := d.mu.snapshots.count(); v > 0 { 1689 err = firstError(err, errors.Errorf("leaked snapshots: %d open snapshots on DB %p", v, d)) 1690 } 1691 1692 return err 1693 } 1694 1695 // Compact the specified range of keys in the database. 1696 func (d *DB) Compact(start, end []byte, parallelize bool) error { 1697 if err := d.closed.Load(); err != nil { 1698 panic(err) 1699 } 1700 if d.opts.ReadOnly { 1701 return ErrReadOnly 1702 } 1703 if d.cmp(start, end) >= 0 { 1704 return errors.Errorf("Compact start %s is not less than end %s", 1705 d.opts.Comparer.FormatKey(start), d.opts.Comparer.FormatKey(end)) 1706 } 1707 iStart := base.MakeInternalKey(start, InternalKeySeqNumMax, InternalKeyKindMax) 1708 iEnd := base.MakeInternalKey(end, 0, 0) 1709 m := (&fileMetadata{}).ExtendPointKeyBounds(d.cmp, iStart, iEnd) 1710 meta := []*fileMetadata{m} 1711 1712 d.mu.Lock() 1713 maxLevelWithFiles := 1 1714 cur := d.mu.versions.currentVersion() 1715 for level := 0; level < numLevels; level++ { 1716 overlaps := cur.Overlaps(level, d.cmp, start, end, iEnd.IsExclusiveSentinel()) 1717 if !overlaps.Empty() { 1718 maxLevelWithFiles = level + 1 1719 } 1720 } 1721 1722 keyRanges := make([]internalKeyRange, len(meta)) 1723 for i := range meta { 1724 keyRanges[i] = internalKeyRange{smallest: m.Smallest, largest: m.Largest} 1725 } 1726 // Determine if any memtable overlaps with the compaction range. We wait for 1727 // any such overlap to flush (initiating a flush if necessary). 1728 mem, err := func() (*flushableEntry, error) { 1729 // Check to see if any files overlap with any of the memtables. The queue 1730 // is ordered from oldest to newest with the mutable memtable being the 1731 // last element in the slice. We want to wait for the newest table that 1732 // overlaps. 1733 for i := len(d.mu.mem.queue) - 1; i >= 0; i-- { 1734 mem := d.mu.mem.queue[i] 1735 if ingestMemtableOverlaps(d.cmp, mem, keyRanges) { 1736 var err error 1737 if mem.flushable == d.mu.mem.mutable { 1738 // We have to hold both commitPipeline.mu and DB.mu when calling 1739 // makeRoomForWrite(). Lock order requirements elsewhere force us to 1740 // unlock DB.mu in order to grab commitPipeline.mu first. 1741 d.mu.Unlock() 1742 d.commit.mu.Lock() 1743 d.mu.Lock() 1744 defer d.commit.mu.Unlock() 1745 if mem.flushable == d.mu.mem.mutable { 1746 // Only flush if the active memtable is unchanged. 1747 err = d.makeRoomForWrite(nil) 1748 } 1749 } 1750 mem.flushForced = true 1751 d.maybeScheduleFlush() 1752 return mem, err 1753 } 1754 } 1755 return nil, nil 1756 }() 1757 1758 d.mu.Unlock() 1759 1760 if err != nil { 1761 return err 1762 } 1763 if mem != nil { 1764 <-mem.flushed 1765 } 1766 1767 for level := 0; level < maxLevelWithFiles; { 1768 if err := d.manualCompact( 1769 iStart.UserKey, iEnd.UserKey, level, parallelize); err != nil { 1770 return err 1771 } 1772 level++ 1773 if level == numLevels-1 { 1774 // A manual compaction of the bottommost level occurred. 1775 // There is no next level to try and compact. 1776 break 1777 } 1778 } 1779 return nil 1780 } 1781 1782 func (d *DB) manualCompact(start, end []byte, level int, parallelize bool) error { 1783 d.mu.Lock() 1784 curr := d.mu.versions.currentVersion() 1785 files := curr.Overlaps(level, d.cmp, start, end, false) 1786 if files.Empty() { 1787 d.mu.Unlock() 1788 return nil 1789 } 1790 1791 var compactions []*manualCompaction 1792 if parallelize { 1793 compactions = append(compactions, d.splitManualCompaction(start, end, level)...) 1794 } else { 1795 compactions = append(compactions, &manualCompaction{ 1796 level: level, 1797 done: make(chan error, 1), 1798 start: start, 1799 end: end, 1800 }) 1801 } 1802 d.mu.compact.manual = append(d.mu.compact.manual, compactions...) 1803 d.maybeScheduleCompaction() 1804 d.mu.Unlock() 1805 1806 // Each of the channels is guaranteed to be eventually sent to once. After a 1807 // compaction is possibly picked in d.maybeScheduleCompaction(), either the 1808 // compaction is dropped, executed after being scheduled, or retried later. 1809 // Assuming eventual progress when a compaction is retried, all outcomes send 1810 // a value to the done channel. Since the channels are buffered, it is not 1811 // necessary to read from each channel, and so we can exit early in the event 1812 // of an error. 1813 for _, compaction := range compactions { 1814 if err := <-compaction.done; err != nil { 1815 return err 1816 } 1817 } 1818 return nil 1819 } 1820 1821 // splitManualCompaction splits a manual compaction over [start,end] on level 1822 // such that the resulting compactions have no key overlap. 1823 func (d *DB) splitManualCompaction( 1824 start, end []byte, level int, 1825 ) (splitCompactions []*manualCompaction) { 1826 curr := d.mu.versions.currentVersion() 1827 endLevel := level + 1 1828 baseLevel := d.mu.versions.picker.getBaseLevel() 1829 if level == 0 { 1830 endLevel = baseLevel 1831 } 1832 keyRanges := calculateInuseKeyRanges(curr, d.cmp, level, endLevel, start, end) 1833 for _, keyRange := range keyRanges { 1834 splitCompactions = append(splitCompactions, &manualCompaction{ 1835 level: level, 1836 done: make(chan error, 1), 1837 start: keyRange.Start, 1838 end: keyRange.End, 1839 split: true, 1840 }) 1841 } 1842 return splitCompactions 1843 } 1844 1845 // DownloadSpan is a key range passed to the Download method. 1846 type DownloadSpan struct { 1847 StartKey []byte 1848 // EndKey is exclusive. 1849 EndKey []byte 1850 } 1851 1852 // Download ensures that the LSM does not use any external sstables for the 1853 // given key ranges. It does so by performing appropriate compactions so that 1854 // all external data becomes available locally. 1855 // 1856 // Note that calling this method does not imply that all other compactions stop; 1857 // it simply informs Pebble of a list of spans for which external data should be 1858 // downloaded with high priority. 1859 // 1860 // The method returns once no external sstasbles overlap the given spans, the 1861 // context is canceled, or an error is hit. 1862 // 1863 // TODO(radu): consider passing a priority/impact knob to express how important 1864 // the download is (versus live traffic performance, LSM health). 1865 func (d *DB) Download(ctx context.Context, spans []DownloadSpan) error { 1866 return errors.Errorf("not implemented") 1867 } 1868 1869 // Flush the memtable to stable storage. 1870 func (d *DB) Flush() error { 1871 flushDone, err := d.AsyncFlush() 1872 if err != nil { 1873 return err 1874 } 1875 <-flushDone 1876 return nil 1877 } 1878 1879 // AsyncFlush asynchronously flushes the memtable to stable storage. 1880 // 1881 // If no error is returned, the caller can receive from the returned channel in 1882 // order to wait for the flush to complete. 1883 func (d *DB) AsyncFlush() (<-chan struct{}, error) { 1884 if err := d.closed.Load(); err != nil { 1885 panic(err) 1886 } 1887 if d.opts.ReadOnly { 1888 return nil, ErrReadOnly 1889 } 1890 1891 d.commit.mu.Lock() 1892 defer d.commit.mu.Unlock() 1893 d.mu.Lock() 1894 defer d.mu.Unlock() 1895 flushed := d.mu.mem.queue[len(d.mu.mem.queue)-1].flushed 1896 err := d.makeRoomForWrite(nil) 1897 if err != nil { 1898 return nil, err 1899 } 1900 return flushed, nil 1901 } 1902 1903 // Metrics returns metrics about the database. 1904 func (d *DB) Metrics() *Metrics { 1905 metrics := &Metrics{} 1906 recycledLogsCount, recycledLogSize := d.logRecycler.stats() 1907 1908 d.mu.Lock() 1909 vers := d.mu.versions.currentVersion() 1910 *metrics = d.mu.versions.metrics 1911 metrics.Compact.EstimatedDebt = d.mu.versions.picker.estimatedCompactionDebt(0) 1912 metrics.Compact.InProgressBytes = d.mu.versions.atomicInProgressBytes.Load() 1913 metrics.Compact.NumInProgress = int64(d.mu.compact.compactingCount) 1914 metrics.Compact.MarkedFiles = vers.Stats.MarkedForCompaction 1915 metrics.Compact.Duration = d.mu.compact.duration 1916 for c := range d.mu.compact.inProgress { 1917 if c.kind != compactionKindFlush { 1918 metrics.Compact.Duration += d.timeNow().Sub(c.beganAt) 1919 } 1920 } 1921 1922 for _, m := range d.mu.mem.queue { 1923 metrics.MemTable.Size += m.totalBytes() 1924 } 1925 metrics.Snapshots.Count = d.mu.snapshots.count() 1926 if metrics.Snapshots.Count > 0 { 1927 metrics.Snapshots.EarliestSeqNum = d.mu.snapshots.earliest() 1928 } 1929 metrics.Snapshots.PinnedKeys = d.mu.snapshots.cumulativePinnedCount 1930 metrics.Snapshots.PinnedSize = d.mu.snapshots.cumulativePinnedSize 1931 metrics.MemTable.Count = int64(len(d.mu.mem.queue)) 1932 metrics.MemTable.ZombieCount = d.memTableCount.Load() - metrics.MemTable.Count 1933 metrics.MemTable.ZombieSize = uint64(d.memTableReserved.Load()) - metrics.MemTable.Size 1934 metrics.WAL.ObsoleteFiles = int64(recycledLogsCount) 1935 metrics.WAL.ObsoletePhysicalSize = recycledLogSize 1936 metrics.WAL.Size = d.logSize.Load() 1937 // The current WAL size (d.atomic.logSize) is the current logical size, 1938 // which may be less than the WAL's physical size if it was recycled. 1939 // The file sizes in d.mu.log.queue are updated to the physical size 1940 // during WAL rotation. Use the larger of the two for the current WAL. All 1941 // the previous WALs's fileSizes in d.mu.log.queue are already updated. 1942 metrics.WAL.PhysicalSize = metrics.WAL.Size 1943 if len(d.mu.log.queue) > 0 && metrics.WAL.PhysicalSize < d.mu.log.queue[len(d.mu.log.queue)-1].fileSize { 1944 metrics.WAL.PhysicalSize = d.mu.log.queue[len(d.mu.log.queue)-1].fileSize 1945 } 1946 for i, n := 0, len(d.mu.log.queue)-1; i < n; i++ { 1947 metrics.WAL.PhysicalSize += d.mu.log.queue[i].fileSize 1948 } 1949 1950 metrics.WAL.BytesIn = d.mu.log.bytesIn // protected by d.mu 1951 for i, n := 0, len(d.mu.mem.queue)-1; i < n; i++ { 1952 metrics.WAL.Size += d.mu.mem.queue[i].logSize 1953 } 1954 metrics.WAL.BytesWritten = metrics.Levels[0].BytesIn + metrics.WAL.Size 1955 if p := d.mu.versions.picker; p != nil { 1956 compactions := d.getInProgressCompactionInfoLocked(nil) 1957 for level, score := range p.getScores(compactions) { 1958 metrics.Levels[level].Score = score 1959 } 1960 } 1961 metrics.Table.ZombieCount = int64(len(d.mu.versions.zombieTables)) 1962 for _, size := range d.mu.versions.zombieTables { 1963 metrics.Table.ZombieSize += size 1964 } 1965 metrics.private.optionsFileSize = d.optionsFileSize 1966 1967 // TODO(jackson): Consider making these metrics optional. 1968 metrics.Keys.RangeKeySetsCount = countRangeKeySetFragments(vers) 1969 metrics.Keys.TombstoneCount = countTombstones(vers) 1970 1971 d.mu.versions.logLock() 1972 metrics.private.manifestFileSize = uint64(d.mu.versions.manifest.Size()) 1973 metrics.Table.BackingTableCount = uint64(len(d.mu.versions.backingState.fileBackingMap)) 1974 metrics.Table.BackingTableSize = d.mu.versions.backingState.fileBackingSize 1975 if invariants.Enabled { 1976 var totalSize uint64 1977 for _, backing := range d.mu.versions.backingState.fileBackingMap { 1978 totalSize += backing.Size 1979 } 1980 if totalSize != metrics.Table.BackingTableSize { 1981 panic("pebble: invalid backing table size accounting") 1982 } 1983 } 1984 d.mu.versions.logUnlock() 1985 1986 metrics.LogWriter.FsyncLatency = d.mu.log.metrics.fsyncLatency 1987 if err := metrics.LogWriter.Merge(&d.mu.log.metrics.LogWriterMetrics); err != nil { 1988 d.opts.Logger.Infof("metrics error: %s", err) 1989 } 1990 metrics.Flush.WriteThroughput = d.mu.compact.flushWriteThroughput 1991 if d.mu.compact.flushing { 1992 metrics.Flush.NumInProgress = 1 1993 } 1994 for i := 0; i < numLevels; i++ { 1995 metrics.Levels[i].Additional.ValueBlocksSize = valueBlocksSizeForLevel(vers, i) 1996 } 1997 1998 d.mu.Unlock() 1999 2000 metrics.BlockCache = d.opts.Cache.Metrics() 2001 metrics.TableCache, metrics.Filter = d.tableCache.metrics() 2002 metrics.TableIters = int64(d.tableCache.iterCount()) 2003 2004 metrics.SecondaryCacheMetrics = d.objProvider.Metrics() 2005 2006 metrics.Uptime = d.timeNow().Sub(d.openedAt) 2007 2008 return metrics 2009 } 2010 2011 // sstablesOptions hold the optional parameters to retrieve TableInfo for all sstables. 2012 type sstablesOptions struct { 2013 // set to true will return the sstable properties in TableInfo 2014 withProperties bool 2015 2016 // if set, return sstables that overlap the key range (end-exclusive) 2017 start []byte 2018 end []byte 2019 2020 withApproximateSpanBytes bool 2021 } 2022 2023 // SSTablesOption set optional parameter used by `DB.SSTables`. 2024 type SSTablesOption func(*sstablesOptions) 2025 2026 // WithProperties enable return sstable properties in each TableInfo. 2027 // 2028 // NOTE: if most of the sstable properties need to be read from disk, 2029 // this options may make method `SSTables` quite slow. 2030 func WithProperties() SSTablesOption { 2031 return func(opt *sstablesOptions) { 2032 opt.withProperties = true 2033 } 2034 } 2035 2036 // WithKeyRangeFilter ensures returned sstables overlap start and end (end-exclusive) 2037 // if start and end are both nil these properties have no effect. 2038 func WithKeyRangeFilter(start, end []byte) SSTablesOption { 2039 return func(opt *sstablesOptions) { 2040 opt.end = end 2041 opt.start = start 2042 } 2043 } 2044 2045 // WithApproximateSpanBytes enables capturing the approximate number of bytes that 2046 // overlap the provided key span for each sstable. 2047 // NOTE: this option can only be used with WithKeyRangeFilter and WithProperties 2048 // provided. 2049 func WithApproximateSpanBytes() SSTablesOption { 2050 return func(opt *sstablesOptions) { 2051 opt.withApproximateSpanBytes = true 2052 } 2053 } 2054 2055 // BackingType denotes the type of storage backing a given sstable. 2056 type BackingType int 2057 2058 const ( 2059 // BackingTypeLocal denotes an sstable stored on local disk according to the 2060 // objprovider. This file is completely owned by us. 2061 BackingTypeLocal BackingType = iota 2062 // BackingTypeShared denotes an sstable stored on shared storage, created 2063 // by this Pebble instance and possibly shared by other Pebble instances. 2064 // These types of files have lifecycle managed by Pebble. 2065 BackingTypeShared 2066 // BackingTypeSharedForeign denotes an sstable stored on shared storage, 2067 // created by a Pebble instance other than this one. These types of files have 2068 // lifecycle managed by Pebble. 2069 BackingTypeSharedForeign 2070 // BackingTypeExternal denotes an sstable stored on external storage, 2071 // not owned by any Pebble instance and with no refcounting/cleanup methods 2072 // or lifecycle management. An example of an external file is a file restored 2073 // from a backup. 2074 BackingTypeExternal 2075 ) 2076 2077 // SSTableInfo export manifest.TableInfo with sstable.Properties alongside 2078 // other file backing info. 2079 type SSTableInfo struct { 2080 manifest.TableInfo 2081 // Virtual indicates whether the sstable is virtual. 2082 Virtual bool 2083 // BackingSSTNum is the file number associated with backing sstable which 2084 // backs the sstable associated with this SSTableInfo. If Virtual is false, 2085 // then BackingSSTNum == FileNum. 2086 BackingSSTNum base.FileNum 2087 // BackingType is the type of storage backing this sstable. 2088 BackingType BackingType 2089 // Locator is the remote.Locator backing this sstable, if the backing type is 2090 // not BackingTypeLocal. 2091 Locator remote.Locator 2092 2093 // Properties is the sstable properties of this table. If Virtual is true, 2094 // then the Properties are associated with the backing sst. 2095 Properties *sstable.Properties 2096 } 2097 2098 // SSTables retrieves the current sstables. The returned slice is indexed by 2099 // level and each level is indexed by the position of the sstable within the 2100 // level. Note that this information may be out of date due to concurrent 2101 // flushes and compactions. 2102 func (d *DB) SSTables(opts ...SSTablesOption) ([][]SSTableInfo, error) { 2103 opt := &sstablesOptions{} 2104 for _, fn := range opts { 2105 fn(opt) 2106 } 2107 2108 if opt.withApproximateSpanBytes && !opt.withProperties { 2109 return nil, errors.Errorf("Cannot use WithApproximateSpanBytes without WithProperties option.") 2110 } 2111 if opt.withApproximateSpanBytes && (opt.start == nil || opt.end == nil) { 2112 return nil, errors.Errorf("Cannot use WithApproximateSpanBytes without WithKeyRangeFilter option.") 2113 } 2114 2115 // Grab and reference the current readState. 2116 readState := d.loadReadState() 2117 defer readState.unref() 2118 2119 // TODO(peter): This is somewhat expensive, especially on a large 2120 // database. It might be worthwhile to unify TableInfo and FileMetadata and 2121 // then we could simply return current.Files. Note that RocksDB is doing 2122 // something similar to the current code, so perhaps it isn't too bad. 2123 srcLevels := readState.current.Levels 2124 var totalTables int 2125 for i := range srcLevels { 2126 totalTables += srcLevels[i].Len() 2127 } 2128 2129 destTables := make([]SSTableInfo, totalTables) 2130 destLevels := make([][]SSTableInfo, len(srcLevels)) 2131 for i := range destLevels { 2132 iter := srcLevels[i].Iter() 2133 j := 0 2134 for m := iter.First(); m != nil; m = iter.Next() { 2135 if opt.start != nil && opt.end != nil && !m.Overlaps(d.opts.Comparer.Compare, opt.start, opt.end, true /* exclusive end */) { 2136 continue 2137 } 2138 destTables[j] = SSTableInfo{TableInfo: m.TableInfo()} 2139 if opt.withProperties { 2140 p, err := d.tableCache.getTableProperties( 2141 m, 2142 ) 2143 if err != nil { 2144 return nil, err 2145 } 2146 destTables[j].Properties = p 2147 } 2148 destTables[j].Virtual = m.Virtual 2149 destTables[j].BackingSSTNum = m.FileBacking.DiskFileNum.FileNum() 2150 objMeta, err := d.objProvider.Lookup(fileTypeTable, m.FileBacking.DiskFileNum) 2151 if err != nil { 2152 return nil, err 2153 } 2154 if objMeta.IsRemote() { 2155 if objMeta.IsShared() { 2156 if d.objProvider.IsSharedForeign(objMeta) { 2157 destTables[j].BackingType = BackingTypeSharedForeign 2158 } else { 2159 destTables[j].BackingType = BackingTypeShared 2160 } 2161 } else { 2162 destTables[j].BackingType = BackingTypeExternal 2163 } 2164 destTables[j].Locator = objMeta.Remote.Locator 2165 } else { 2166 destTables[j].BackingType = BackingTypeLocal 2167 } 2168 2169 if opt.withApproximateSpanBytes { 2170 var spanBytes uint64 2171 if m.ContainedWithinSpan(d.opts.Comparer.Compare, opt.start, opt.end) { 2172 spanBytes = m.Size 2173 } else { 2174 size, err := d.tableCache.estimateSize(m, opt.start, opt.end) 2175 if err != nil { 2176 return nil, err 2177 } 2178 spanBytes = size 2179 } 2180 propertiesCopy := *destTables[j].Properties 2181 2182 // Deep copy user properties so approximate span bytes can be added. 2183 propertiesCopy.UserProperties = make(map[string]string, len(destTables[j].Properties.UserProperties)+1) 2184 for k, v := range destTables[j].Properties.UserProperties { 2185 propertiesCopy.UserProperties[k] = v 2186 } 2187 propertiesCopy.UserProperties["approximate-span-bytes"] = strconv.FormatUint(spanBytes, 10) 2188 destTables[j].Properties = &propertiesCopy 2189 } 2190 j++ 2191 } 2192 destLevels[i] = destTables[:j] 2193 destTables = destTables[j:] 2194 } 2195 2196 return destLevels, nil 2197 } 2198 2199 // EstimateDiskUsage returns the estimated filesystem space used in bytes for 2200 // storing the range `[start, end]`. The estimation is computed as follows: 2201 // 2202 // - For sstables fully contained in the range the whole file size is included. 2203 // - For sstables partially contained in the range the overlapping data block sizes 2204 // are included. Even if a data block partially overlaps, or we cannot determine 2205 // overlap due to abbreviated index keys, the full data block size is included in 2206 // the estimation. Note that unlike fully contained sstables, none of the 2207 // meta-block space is counted for partially overlapped files. 2208 // - For virtual sstables, we use the overlap between start, end and the virtual 2209 // sstable bounds to determine disk usage. 2210 // - There may also exist WAL entries for unflushed keys in this range. This 2211 // estimation currently excludes space used for the range in the WAL. 2212 func (d *DB) EstimateDiskUsage(start, end []byte) (uint64, error) { 2213 bytes, _, _, err := d.EstimateDiskUsageByBackingType(start, end) 2214 return bytes, err 2215 } 2216 2217 // EstimateDiskUsageByBackingType is like EstimateDiskUsage but additionally 2218 // returns the subsets of that size in remote ane external files. 2219 func (d *DB) EstimateDiskUsageByBackingType( 2220 start, end []byte, 2221 ) (totalSize, remoteSize, externalSize uint64, _ error) { 2222 if err := d.closed.Load(); err != nil { 2223 panic(err) 2224 } 2225 if d.opts.Comparer.Compare(start, end) > 0 { 2226 return 0, 0, 0, errors.New("invalid key-range specified (start > end)") 2227 } 2228 2229 // Grab and reference the current readState. This prevents the underlying 2230 // files in the associated version from being deleted if there is a concurrent 2231 // compaction. 2232 readState := d.loadReadState() 2233 defer readState.unref() 2234 2235 for level, files := range readState.current.Levels { 2236 iter := files.Iter() 2237 if level > 0 { 2238 // We can only use `Overlaps` to restrict `files` at L1+ since at L0 it 2239 // expands the range iteratively until it has found a set of files that 2240 // do not overlap any other L0 files outside that set. 2241 overlaps := readState.current.Overlaps(level, d.opts.Comparer.Compare, start, end, false /* exclusiveEnd */) 2242 iter = overlaps.Iter() 2243 } 2244 for file := iter.First(); file != nil; file = iter.Next() { 2245 if d.opts.Comparer.Compare(start, file.Smallest.UserKey) <= 0 && 2246 d.opts.Comparer.Compare(file.Largest.UserKey, end) <= 0 { 2247 // The range fully contains the file, so skip looking it up in 2248 // table cache/looking at its indexes, and add the full file size. 2249 meta, err := d.objProvider.Lookup(fileTypeTable, file.FileBacking.DiskFileNum) 2250 if err != nil { 2251 return 0, 0, 0, err 2252 } 2253 if meta.IsRemote() { 2254 remoteSize += file.Size 2255 if meta.Remote.CleanupMethod == objstorage.SharedNoCleanup { 2256 externalSize += file.Size 2257 } 2258 } 2259 totalSize += file.Size 2260 } else if d.opts.Comparer.Compare(file.Smallest.UserKey, end) <= 0 && 2261 d.opts.Comparer.Compare(start, file.Largest.UserKey) <= 0 { 2262 var size uint64 2263 var err error 2264 if file.Virtual { 2265 err = d.tableCache.withVirtualReader( 2266 file.VirtualMeta(), 2267 func(r sstable.VirtualReader) (err error) { 2268 size, err = r.EstimateDiskUsage(start, end) 2269 return err 2270 }, 2271 ) 2272 } else { 2273 err = d.tableCache.withReader( 2274 file.PhysicalMeta(), 2275 func(r *sstable.Reader) (err error) { 2276 size, err = r.EstimateDiskUsage(start, end) 2277 return err 2278 }, 2279 ) 2280 } 2281 if err != nil { 2282 return 0, 0, 0, err 2283 } 2284 meta, err := d.objProvider.Lookup(fileTypeTable, file.FileBacking.DiskFileNum) 2285 if err != nil { 2286 return 0, 0, 0, err 2287 } 2288 if meta.IsRemote() { 2289 remoteSize += size 2290 if meta.Remote.CleanupMethod == objstorage.SharedNoCleanup { 2291 externalSize += size 2292 } 2293 } 2294 totalSize += size 2295 } 2296 } 2297 } 2298 return totalSize, remoteSize, externalSize, nil 2299 } 2300 2301 func (d *DB) walPreallocateSize() int { 2302 // Set the WAL preallocate size to 110% of the memtable size. Note that there 2303 // is a bit of apples and oranges in units here as the memtabls size 2304 // corresponds to the memory usage of the memtable while the WAL size is the 2305 // size of the batches (plus overhead) stored in the WAL. 2306 // 2307 // TODO(peter): 110% of the memtable size is quite hefty for a block 2308 // size. This logic is taken from GetWalPreallocateBlockSize in 2309 // RocksDB. Could a smaller preallocation block size be used? 2310 size := d.opts.MemTableSize 2311 size = (size / 10) + size 2312 return int(size) 2313 } 2314 2315 func (d *DB) newMemTable(logNum FileNum, logSeqNum uint64) (*memTable, *flushableEntry) { 2316 size := d.mu.mem.nextSize 2317 if d.mu.mem.nextSize < d.opts.MemTableSize { 2318 d.mu.mem.nextSize *= 2 2319 if d.mu.mem.nextSize > d.opts.MemTableSize { 2320 d.mu.mem.nextSize = d.opts.MemTableSize 2321 } 2322 } 2323 2324 memtblOpts := memTableOptions{ 2325 Options: d.opts, 2326 logSeqNum: logSeqNum, 2327 } 2328 2329 // Before attempting to allocate a new memtable, check if there's one 2330 // available for recycling in memTableRecycle. Large contiguous allocations 2331 // can be costly as fragmentation makes it more difficult to find a large 2332 // contiguous free space. We've observed 64MB allocations taking 10ms+. 2333 // 2334 // To reduce these costly allocations, up to 1 obsolete memtable is stashed 2335 // in `d.memTableRecycle` to allow a future memtable rotation to reuse 2336 // existing memory. 2337 var mem *memTable 2338 mem = d.memTableRecycle.Swap(nil) 2339 if mem != nil && uint64(len(mem.arenaBuf)) != size { 2340 d.freeMemTable(mem) 2341 mem = nil 2342 } 2343 if mem != nil { 2344 // Carry through the existing buffer and memory reservation. 2345 memtblOpts.arenaBuf = mem.arenaBuf 2346 memtblOpts.releaseAccountingReservation = mem.releaseAccountingReservation 2347 } else { 2348 mem = new(memTable) 2349 memtblOpts.arenaBuf = manual.New(int(size)) 2350 memtblOpts.releaseAccountingReservation = d.opts.Cache.Reserve(int(size)) 2351 d.memTableCount.Add(1) 2352 d.memTableReserved.Add(int64(size)) 2353 2354 // Note: this is a no-op if invariants are disabled or race is enabled. 2355 invariants.SetFinalizer(mem, checkMemTable) 2356 } 2357 mem.init(memtblOpts) 2358 2359 entry := d.newFlushableEntry(mem, logNum, logSeqNum) 2360 entry.releaseMemAccounting = func() { 2361 // If the user leaks iterators, we may be releasing the memtable after 2362 // the DB is already closed. In this case, we want to just release the 2363 // memory because DB.Close won't come along to free it for us. 2364 if err := d.closed.Load(); err != nil { 2365 d.freeMemTable(mem) 2366 return 2367 } 2368 2369 // The next memtable allocation might be able to reuse this memtable. 2370 // Stash it on d.memTableRecycle. 2371 if unusedMem := d.memTableRecycle.Swap(mem); unusedMem != nil { 2372 // There was already a memtable waiting to be recycled. We're now 2373 // responsible for freeing it. 2374 d.freeMemTable(unusedMem) 2375 } 2376 } 2377 return mem, entry 2378 } 2379 2380 func (d *DB) freeMemTable(m *memTable) { 2381 d.memTableCount.Add(-1) 2382 d.memTableReserved.Add(-int64(len(m.arenaBuf))) 2383 m.free() 2384 } 2385 2386 func (d *DB) newFlushableEntry(f flushable, logNum FileNum, logSeqNum uint64) *flushableEntry { 2387 fe := &flushableEntry{ 2388 flushable: f, 2389 flushed: make(chan struct{}), 2390 logNum: logNum, 2391 logSeqNum: logSeqNum, 2392 deleteFn: d.mu.versions.addObsolete, 2393 deleteFnLocked: d.mu.versions.addObsoleteLocked, 2394 } 2395 fe.readerRefs.Store(1) 2396 return fe 2397 } 2398 2399 // makeRoomForWrite ensures that the memtable has room to hold the contents of 2400 // Batch. It reserves the space in the memtable and adds a reference to the 2401 // memtable. The caller must later ensure that the memtable is unreferenced. If 2402 // the memtable is full, or a nil Batch is provided, the current memtable is 2403 // rotated (marked as immutable) and a new mutable memtable is allocated. This 2404 // memtable rotation also causes a log rotation. 2405 // 2406 // Both DB.mu and commitPipeline.mu must be held by the caller. Note that DB.mu 2407 // may be released and reacquired. 2408 func (d *DB) makeRoomForWrite(b *Batch) error { 2409 if b != nil && b.ingestedSSTBatch { 2410 panic("pebble: invalid function call") 2411 } 2412 2413 force := b == nil || b.flushable != nil 2414 stalled := false 2415 for { 2416 if b != nil && b.flushable == nil { 2417 err := d.mu.mem.mutable.prepare(b) 2418 if err != arenaskl.ErrArenaFull { 2419 if stalled { 2420 d.opts.EventListener.WriteStallEnd() 2421 } 2422 return err 2423 } 2424 } else if !force { 2425 if stalled { 2426 d.opts.EventListener.WriteStallEnd() 2427 } 2428 return nil 2429 } 2430 // force || err == ErrArenaFull, so we need to rotate the current memtable. 2431 { 2432 var size uint64 2433 for i := range d.mu.mem.queue { 2434 size += d.mu.mem.queue[i].totalBytes() 2435 } 2436 if size >= uint64(d.opts.MemTableStopWritesThreshold)*d.opts.MemTableSize { 2437 // We have filled up the current memtable, but already queued memtables 2438 // are still flushing, so we wait. 2439 if !stalled { 2440 stalled = true 2441 d.opts.EventListener.WriteStallBegin(WriteStallBeginInfo{ 2442 Reason: "memtable count limit reached", 2443 }) 2444 } 2445 now := time.Now() 2446 d.mu.compact.cond.Wait() 2447 if b != nil { 2448 b.commitStats.MemTableWriteStallDuration += time.Since(now) 2449 } 2450 continue 2451 } 2452 } 2453 l0ReadAmp := d.mu.versions.currentVersion().L0Sublevels.ReadAmplification() 2454 if l0ReadAmp >= d.opts.L0StopWritesThreshold { 2455 // There are too many level-0 files, so we wait. 2456 if !stalled { 2457 stalled = true 2458 d.opts.EventListener.WriteStallBegin(WriteStallBeginInfo{ 2459 Reason: "L0 file count limit exceeded", 2460 }) 2461 } 2462 now := time.Now() 2463 d.mu.compact.cond.Wait() 2464 if b != nil { 2465 b.commitStats.L0ReadAmpWriteStallDuration += time.Since(now) 2466 } 2467 continue 2468 } 2469 2470 var newLogNum base.FileNum 2471 var prevLogSize uint64 2472 if !d.opts.DisableWAL { 2473 now := time.Now() 2474 newLogNum, prevLogSize = d.rotateWAL() 2475 if b != nil { 2476 b.commitStats.WALRotationDuration += time.Since(now) 2477 } 2478 } 2479 2480 immMem := d.mu.mem.mutable 2481 imm := d.mu.mem.queue[len(d.mu.mem.queue)-1] 2482 imm.logSize = prevLogSize 2483 imm.flushForced = imm.flushForced || (b == nil) 2484 2485 // If we are manually flushing and we used less than half of the bytes in 2486 // the memtable, don't increase the size for the next memtable. This 2487 // reduces memtable memory pressure when an application is frequently 2488 // manually flushing. 2489 if (b == nil) && uint64(immMem.availBytes()) > immMem.totalBytes()/2 { 2490 d.mu.mem.nextSize = immMem.totalBytes() 2491 } 2492 2493 if b != nil && b.flushable != nil { 2494 // The batch is too large to fit in the memtable so add it directly to 2495 // the immutable queue. The flushable batch is associated with the same 2496 // log as the immutable memtable, but logically occurs after it in 2497 // seqnum space. We ensure while flushing that the flushable batch 2498 // is flushed along with the previous memtable in the flushable 2499 // queue. See the top level comment in DB.flush1 to learn how this 2500 // is ensured. 2501 // 2502 // See DB.commitWrite for the special handling of log writes for large 2503 // batches. In particular, the large batch has already written to 2504 // imm.logNum. 2505 entry := d.newFlushableEntry(b.flushable, imm.logNum, b.SeqNum()) 2506 // The large batch is by definition large. Reserve space from the cache 2507 // for it until it is flushed. 2508 entry.releaseMemAccounting = d.opts.Cache.Reserve(int(b.flushable.totalBytes())) 2509 d.mu.mem.queue = append(d.mu.mem.queue, entry) 2510 } 2511 2512 var logSeqNum uint64 2513 if b != nil { 2514 logSeqNum = b.SeqNum() 2515 if b.flushable != nil { 2516 logSeqNum += uint64(b.Count()) 2517 } 2518 } else { 2519 logSeqNum = d.mu.versions.logSeqNum.Load() 2520 } 2521 d.rotateMemtable(newLogNum, logSeqNum, immMem) 2522 force = false 2523 } 2524 } 2525 2526 // Both DB.mu and commitPipeline.mu must be held by the caller. 2527 func (d *DB) rotateMemtable(newLogNum FileNum, logSeqNum uint64, prev *memTable) { 2528 // Create a new memtable, scheduling the previous one for flushing. We do 2529 // this even if the previous memtable was empty because the DB.Flush 2530 // mechanism is dependent on being able to wait for the empty memtable to 2531 // flush. We can't just mark the empty memtable as flushed here because we 2532 // also have to wait for all previous immutable tables to 2533 // flush. Additionally, the memtable is tied to particular WAL file and we 2534 // want to go through the flush path in order to recycle that WAL file. 2535 // 2536 // NB: newLogNum corresponds to the WAL that contains mutations that are 2537 // present in the new memtable. When immutable memtables are flushed to 2538 // disk, a VersionEdit will be created telling the manifest the minimum 2539 // unflushed log number (which will be the next one in d.mu.mem.mutable 2540 // that was not flushed). 2541 // 2542 // NB: prev should be the current mutable memtable. 2543 var entry *flushableEntry 2544 d.mu.mem.mutable, entry = d.newMemTable(newLogNum, logSeqNum) 2545 d.mu.mem.queue = append(d.mu.mem.queue, entry) 2546 // d.logSize tracks the log size of the WAL file corresponding to the most 2547 // recent flushable. The log size of the previous mutable memtable no longer 2548 // applies to the current mutable memtable. 2549 // 2550 // It's tempting to perform this update in rotateWAL, but that would not be 2551 // atomic with the enqueue of the new flushable. A call to DB.Metrics() 2552 // could acquire DB.mu after the WAL has been rotated but before the new 2553 // memtable has been appended; this would result in omitting the log size of 2554 // the most recent flushable. 2555 d.logSize.Store(0) 2556 d.updateReadStateLocked(nil) 2557 if prev.writerUnref() { 2558 d.maybeScheduleFlush() 2559 } 2560 } 2561 2562 // rotateWAL creates a new write-ahead log, possibly recycling a previous WAL's 2563 // files. It returns the file number assigned to the new WAL, and the size of 2564 // the previous WAL file. 2565 // 2566 // Both DB.mu and commitPipeline.mu must be held by the caller. Note that DB.mu 2567 // may be released and reacquired. 2568 func (d *DB) rotateWAL() (newLogNum FileNum, prevLogSize uint64) { 2569 if d.opts.DisableWAL { 2570 panic("pebble: invalid function call") 2571 } 2572 2573 jobID := d.mu.nextJobID 2574 d.mu.nextJobID++ 2575 newLogNum = d.mu.versions.getNextFileNum() 2576 2577 prevLogSize = uint64(d.mu.log.Size()) 2578 2579 // The previous log may have grown past its original physical 2580 // size. Update its file size in the queue so we have a proper 2581 // accounting of its file size. 2582 if d.mu.log.queue[len(d.mu.log.queue)-1].fileSize < prevLogSize { 2583 d.mu.log.queue[len(d.mu.log.queue)-1].fileSize = prevLogSize 2584 } 2585 d.mu.Unlock() 2586 2587 var err error 2588 // Close the previous log first. This writes an EOF trailer 2589 // signifying the end of the file and syncs it to disk. We must 2590 // close the previous log before linking the new log file, 2591 // otherwise a crash could leave both logs with unclean tails, and 2592 // Open will treat the previous log as corrupt. 2593 err = d.mu.log.LogWriter.Close() 2594 metrics := d.mu.log.LogWriter.Metrics() 2595 d.mu.Lock() 2596 if err := d.mu.log.metrics.Merge(metrics); err != nil { 2597 d.opts.Logger.Infof("metrics error: %s", err) 2598 } 2599 d.mu.Unlock() 2600 2601 newLogName := base.MakeFilepath(d.opts.FS, d.walDirname, fileTypeLog, newLogNum.DiskFileNum()) 2602 2603 // Try to use a recycled log file. Recycling log files is an important 2604 // performance optimization as it is faster to sync a file that has 2605 // already been written, than one which is being written for the first 2606 // time. This is due to the need to sync file metadata when a file is 2607 // being written for the first time. Note this is true even if file 2608 // preallocation is performed (e.g. fallocate). 2609 var recycleLog fileInfo 2610 var recycleOK bool 2611 var newLogFile vfs.File 2612 if err == nil { 2613 recycleLog, recycleOK = d.logRecycler.peek() 2614 if recycleOK { 2615 recycleLogName := base.MakeFilepath(d.opts.FS, d.walDirname, fileTypeLog, recycleLog.fileNum) 2616 newLogFile, err = d.opts.FS.ReuseForWrite(recycleLogName, newLogName) 2617 base.MustExist(d.opts.FS, newLogName, d.opts.Logger, err) 2618 } else { 2619 newLogFile, err = d.opts.FS.Create(newLogName) 2620 base.MustExist(d.opts.FS, newLogName, d.opts.Logger, err) 2621 } 2622 } 2623 2624 var newLogSize uint64 2625 if err == nil && recycleOK { 2626 // Figure out the recycled WAL size. This Stat is necessary 2627 // because ReuseForWrite's contract allows for removing the 2628 // old file and creating a new one. We don't know whether the 2629 // WAL was actually recycled. 2630 // TODO(jackson): Adding a boolean to the ReuseForWrite return 2631 // value indicating whether or not the file was actually 2632 // reused would allow us to skip the stat and use 2633 // recycleLog.fileSize. 2634 var finfo os.FileInfo 2635 finfo, err = newLogFile.Stat() 2636 if err == nil { 2637 newLogSize = uint64(finfo.Size()) 2638 } 2639 } 2640 2641 if err == nil { 2642 // TODO(peter): RocksDB delays sync of the parent directory until the 2643 // first time the log is synced. Is that worthwhile? 2644 err = d.walDir.Sync() 2645 } 2646 2647 if err != nil && newLogFile != nil { 2648 newLogFile.Close() 2649 } else if err == nil { 2650 newLogFile = vfs.NewSyncingFile(newLogFile, vfs.SyncingFileOptions{ 2651 NoSyncOnClose: d.opts.NoSyncOnClose, 2652 BytesPerSync: d.opts.WALBytesPerSync, 2653 PreallocateSize: d.walPreallocateSize(), 2654 }) 2655 } 2656 2657 if recycleOK { 2658 err = firstError(err, d.logRecycler.pop(recycleLog.fileNum.FileNum())) 2659 } 2660 2661 d.opts.EventListener.WALCreated(WALCreateInfo{ 2662 JobID: jobID, 2663 Path: newLogName, 2664 FileNum: newLogNum, 2665 RecycledFileNum: recycleLog.fileNum.FileNum(), 2666 Err: err, 2667 }) 2668 2669 d.mu.Lock() 2670 2671 d.mu.versions.metrics.WAL.Files++ 2672 2673 if err != nil { 2674 // TODO(peter): avoid chewing through file numbers in a tight loop if there 2675 // is an error here. 2676 // 2677 // What to do here? Stumbling on doesn't seem worthwhile. If we failed to 2678 // close the previous log it is possible we lost a write. 2679 panic(err) 2680 } 2681 2682 d.mu.log.queue = append(d.mu.log.queue, fileInfo{fileNum: newLogNum.DiskFileNum(), fileSize: newLogSize}) 2683 d.mu.log.LogWriter = record.NewLogWriter(newLogFile, newLogNum, record.LogWriterConfig{ 2684 WALFsyncLatency: d.mu.log.metrics.fsyncLatency, 2685 WALMinSyncInterval: d.opts.WALMinSyncInterval, 2686 QueueSemChan: d.commit.logSyncQSem, 2687 }) 2688 if d.mu.log.registerLogWriterForTesting != nil { 2689 d.mu.log.registerLogWriterForTesting(d.mu.log.LogWriter) 2690 } 2691 2692 return 2693 } 2694 2695 func (d *DB) getEarliestUnflushedSeqNumLocked() uint64 { 2696 seqNum := InternalKeySeqNumMax 2697 for i := range d.mu.mem.queue { 2698 logSeqNum := d.mu.mem.queue[i].logSeqNum 2699 if seqNum > logSeqNum { 2700 seqNum = logSeqNum 2701 } 2702 } 2703 return seqNum 2704 } 2705 2706 func (d *DB) getInProgressCompactionInfoLocked(finishing *compaction) (rv []compactionInfo) { 2707 for c := range d.mu.compact.inProgress { 2708 if len(c.flushing) == 0 && (finishing == nil || c != finishing) { 2709 info := compactionInfo{ 2710 versionEditApplied: c.versionEditApplied, 2711 inputs: c.inputs, 2712 smallest: c.smallest, 2713 largest: c.largest, 2714 outputLevel: -1, 2715 } 2716 if c.outputLevel != nil { 2717 info.outputLevel = c.outputLevel.level 2718 } 2719 rv = append(rv, info) 2720 } 2721 } 2722 return 2723 } 2724 2725 func inProgressL0Compactions(inProgress []compactionInfo) []manifest.L0Compaction { 2726 var compactions []manifest.L0Compaction 2727 for _, info := range inProgress { 2728 // Skip in-progress compactions that have already committed; the L0 2729 // sublevels initialization code requires the set of in-progress 2730 // compactions to be consistent with the current version. Compactions 2731 // with versionEditApplied=true are already applied to the current 2732 // version and but are performing cleanup without the database mutex. 2733 if info.versionEditApplied { 2734 continue 2735 } 2736 l0 := false 2737 for _, cl := range info.inputs { 2738 l0 = l0 || cl.level == 0 2739 } 2740 if !l0 { 2741 continue 2742 } 2743 compactions = append(compactions, manifest.L0Compaction{ 2744 Smallest: info.smallest, 2745 Largest: info.largest, 2746 IsIntraL0: info.outputLevel == 0, 2747 }) 2748 } 2749 return compactions 2750 } 2751 2752 // firstError returns the first non-nil error of err0 and err1, or nil if both 2753 // are nil. 2754 func firstError(err0, err1 error) error { 2755 if err0 != nil { 2756 return err0 2757 } 2758 return err1 2759 } 2760 2761 // SetCreatorID sets the CreatorID which is needed in order to use shared objects. 2762 // Remote object usage is disabled until this method is called the first time. 2763 // Once set, the Creator ID is persisted and cannot change. 2764 // 2765 // Does nothing if SharedStorage was not set in the options when the DB was 2766 // opened or if the DB is in read-only mode. 2767 func (d *DB) SetCreatorID(creatorID uint64) error { 2768 if d.opts.Experimental.RemoteStorage == nil || d.opts.ReadOnly { 2769 return nil 2770 } 2771 return d.objProvider.SetCreatorID(objstorage.CreatorID(creatorID)) 2772 } 2773 2774 // KeyStatistics keeps track of the number of keys that have been pinned by a 2775 // snapshot as well as counts of the different key kinds in the lsm. 2776 // 2777 // One way of using the accumulated stats, when we only have sets and dels, 2778 // and say the counts are represented as del_count, set_count, 2779 // del_latest_count, set_latest_count, snapshot_pinned_count. 2780 // 2781 // - del_latest_count + set_latest_count is the set of unique user keys 2782 // (unique). 2783 // 2784 // - set_latest_count is the set of live unique user keys (live_unique). 2785 // 2786 // - Garbage is del_count + set_count - live_unique. 2787 // 2788 // - If everything were in the LSM, del_count+set_count-snapshot_pinned_count 2789 // would also be the set of unique user keys (note that 2790 // snapshot_pinned_count is counting something different -- see comment below). 2791 // But snapshot_pinned_count only counts keys in the LSM so the excess here 2792 // must be keys in memtables. 2793 type KeyStatistics struct { 2794 // TODO(sumeer): the SnapshotPinned* are incorrect in that these older 2795 // versions can be in a different level. Either fix the accounting or 2796 // rename these fields. 2797 2798 // SnapshotPinnedKeys represents obsolete keys that cannot be elided during 2799 // a compaction, because they are required by an open snapshot. 2800 SnapshotPinnedKeys int 2801 // SnapshotPinnedKeysBytes is the total number of bytes of all snapshot 2802 // pinned keys. 2803 SnapshotPinnedKeysBytes uint64 2804 // KindsCount is the count for each kind of key. It includes point keys, 2805 // range deletes and range keys. 2806 KindsCount [InternalKeyKindMax + 1]int 2807 // LatestKindsCount is the count for each kind of key when it is the latest 2808 // kind for a user key. It is only populated for point keys. 2809 LatestKindsCount [InternalKeyKindMax + 1]int 2810 } 2811 2812 // LSMKeyStatistics is used by DB.ScanStatistics. 2813 type LSMKeyStatistics struct { 2814 Accumulated KeyStatistics 2815 // Levels contains statistics only for point keys. Range deletions and range keys will 2816 // appear in Accumulated but not Levels. 2817 Levels [numLevels]KeyStatistics 2818 // BytesRead represents the logical, pre-compression size of keys and values read 2819 BytesRead uint64 2820 } 2821 2822 // ScanStatisticsOptions is used by DB.ScanStatistics. 2823 type ScanStatisticsOptions struct { 2824 // LimitBytesPerSecond indicates the number of bytes that are able to be read 2825 // per second using ScanInternal. 2826 // A value of 0 indicates that there is no limit set. 2827 LimitBytesPerSecond int64 2828 } 2829 2830 // ScanStatistics returns the count of different key kinds within the lsm for a 2831 // key span [lower, upper) as well as the number of snapshot keys. 2832 func (d *DB) ScanStatistics( 2833 ctx context.Context, lower, upper []byte, opts ScanStatisticsOptions, 2834 ) (LSMKeyStatistics, error) { 2835 stats := LSMKeyStatistics{} 2836 var prevKey InternalKey 2837 var rateLimitFunc func(key *InternalKey, val LazyValue) error 2838 tb := tokenbucket.TokenBucket{} 2839 2840 if opts.LimitBytesPerSecond != 0 { 2841 // Each "token" roughly corresponds to a byte that was read. 2842 tb.Init(tokenbucket.TokensPerSecond(opts.LimitBytesPerSecond), tokenbucket.Tokens(1024)) 2843 rateLimitFunc = func(key *InternalKey, val LazyValue) error { 2844 return tb.WaitCtx(ctx, tokenbucket.Tokens(key.Size()+val.Len())) 2845 } 2846 } 2847 2848 scanInternalOpts := &scanInternalOptions{ 2849 visitPointKey: func(key *InternalKey, value LazyValue, iterInfo IteratorLevel) error { 2850 // If the previous key is equal to the current point key, the current key was 2851 // pinned by a snapshot. 2852 size := uint64(key.Size()) 2853 kind := key.Kind() 2854 sameKey := d.equal(prevKey.UserKey, key.UserKey) 2855 if iterInfo.Kind == IteratorLevelLSM && sameKey { 2856 stats.Levels[iterInfo.Level].SnapshotPinnedKeys++ 2857 stats.Levels[iterInfo.Level].SnapshotPinnedKeysBytes += size 2858 stats.Accumulated.SnapshotPinnedKeys++ 2859 stats.Accumulated.SnapshotPinnedKeysBytes += size 2860 } 2861 if iterInfo.Kind == IteratorLevelLSM { 2862 stats.Levels[iterInfo.Level].KindsCount[kind]++ 2863 } 2864 if !sameKey { 2865 if iterInfo.Kind == IteratorLevelLSM { 2866 stats.Levels[iterInfo.Level].LatestKindsCount[kind]++ 2867 } 2868 stats.Accumulated.LatestKindsCount[kind]++ 2869 } 2870 2871 stats.Accumulated.KindsCount[kind]++ 2872 prevKey.CopyFrom(*key) 2873 stats.BytesRead += uint64(key.Size() + value.Len()) 2874 return nil 2875 }, 2876 visitRangeDel: func(start, end []byte, seqNum uint64) error { 2877 stats.Accumulated.KindsCount[InternalKeyKindRangeDelete]++ 2878 stats.BytesRead += uint64(len(start) + len(end)) 2879 return nil 2880 }, 2881 visitRangeKey: func(start, end []byte, keys []rangekey.Key) error { 2882 stats.BytesRead += uint64(len(start) + len(end)) 2883 for _, key := range keys { 2884 stats.Accumulated.KindsCount[key.Kind()]++ 2885 stats.BytesRead += uint64(len(key.Value) + len(key.Suffix)) 2886 } 2887 return nil 2888 }, 2889 includeObsoleteKeys: true, 2890 IterOptions: IterOptions{ 2891 KeyTypes: IterKeyTypePointsAndRanges, 2892 LowerBound: lower, 2893 UpperBound: upper, 2894 }, 2895 rateLimitFunc: rateLimitFunc, 2896 } 2897 iter := d.newInternalIter(snapshotIterOpts{}, scanInternalOpts) 2898 defer iter.close() 2899 2900 err := scanInternalImpl(ctx, lower, upper, iter, scanInternalOpts) 2901 2902 if err != nil { 2903 return LSMKeyStatistics{}, err 2904 } 2905 2906 return stats, nil 2907 } 2908 2909 // ObjProvider returns the objstorage.Provider for this database. Meant to be 2910 // used for internal purposes only. 2911 func (d *DB) ObjProvider() objstorage.Provider { 2912 return d.objProvider 2913 } 2914 2915 func (d *DB) checkVirtualBounds(m *fileMetadata) { 2916 if !invariants.Enabled { 2917 return 2918 } 2919 2920 if m.HasPointKeys { 2921 pointIter, rangeDelIter, err := d.newIters(context.TODO(), m, nil, internalIterOpts{}) 2922 if err != nil { 2923 panic(errors.Wrap(err, "pebble: error creating point iterator")) 2924 } 2925 2926 defer pointIter.Close() 2927 if rangeDelIter != nil { 2928 defer rangeDelIter.Close() 2929 } 2930 2931 pointKey, _ := pointIter.First() 2932 var rangeDel *keyspan.Span 2933 if rangeDelIter != nil { 2934 rangeDel = rangeDelIter.First() 2935 } 2936 2937 // Check that the lower bound is tight. 2938 if (rangeDel == nil || d.cmp(rangeDel.SmallestKey().UserKey, m.SmallestPointKey.UserKey) != 0) && 2939 (pointKey == nil || d.cmp(pointKey.UserKey, m.SmallestPointKey.UserKey) != 0) { 2940 panic(errors.Newf("pebble: virtual sstable %s lower point key bound is not tight", m.FileNum)) 2941 } 2942 2943 pointKey, _ = pointIter.Last() 2944 rangeDel = nil 2945 if rangeDelIter != nil { 2946 rangeDel = rangeDelIter.Last() 2947 } 2948 2949 // Check that the upper bound is tight. 2950 if (rangeDel == nil || d.cmp(rangeDel.LargestKey().UserKey, m.LargestPointKey.UserKey) != 0) && 2951 (pointKey == nil || d.cmp(pointKey.UserKey, m.LargestPointKey.UserKey) != 0) { 2952 panic(errors.Newf("pebble: virtual sstable %s upper point key bound is not tight", m.FileNum)) 2953 } 2954 2955 // Check that iterator keys are within bounds. 2956 for key, _ := pointIter.First(); key != nil; key, _ = pointIter.Next() { 2957 if d.cmp(key.UserKey, m.SmallestPointKey.UserKey) < 0 || d.cmp(key.UserKey, m.LargestPointKey.UserKey) > 0 { 2958 panic(errors.Newf("pebble: virtual sstable %s point key %s is not within bounds", m.FileNum, key.UserKey)) 2959 } 2960 } 2961 2962 if rangeDelIter != nil { 2963 for key := rangeDelIter.First(); key != nil; key = rangeDelIter.Next() { 2964 if d.cmp(key.SmallestKey().UserKey, m.SmallestPointKey.UserKey) < 0 { 2965 panic(errors.Newf("pebble: virtual sstable %s point key %s is not within bounds", m.FileNum, key.SmallestKey().UserKey)) 2966 } 2967 2968 if d.cmp(key.LargestKey().UserKey, m.LargestPointKey.UserKey) > 0 { 2969 panic(errors.Newf("pebble: virtual sstable %s point key %s is not within bounds", m.FileNum, key.LargestKey().UserKey)) 2970 } 2971 } 2972 } 2973 } 2974 2975 if !m.HasRangeKeys { 2976 return 2977 } 2978 2979 rangeKeyIter, err := d.tableNewRangeKeyIter(m, keyspan.SpanIterOptions{}) 2980 defer rangeKeyIter.Close() 2981 2982 if err != nil { 2983 panic(errors.Wrap(err, "pebble: error creating range key iterator")) 2984 } 2985 2986 // Check that the lower bound is tight. 2987 if d.cmp(rangeKeyIter.First().SmallestKey().UserKey, m.SmallestRangeKey.UserKey) != 0 { 2988 panic(errors.Newf("pebble: virtual sstable %s lower range key bound is not tight", m.FileNum)) 2989 } 2990 2991 // Check that upper bound is tight. 2992 if d.cmp(rangeKeyIter.Last().LargestKey().UserKey, m.LargestRangeKey.UserKey) != 0 { 2993 panic(errors.Newf("pebble: virtual sstable %s upper range key bound is not tight", m.FileNum)) 2994 } 2995 2996 for key := rangeKeyIter.First(); key != nil; key = rangeKeyIter.Next() { 2997 if d.cmp(key.SmallestKey().UserKey, m.SmallestRangeKey.UserKey) < 0 { 2998 panic(errors.Newf("pebble: virtual sstable %s point key %s is not within bounds", m.FileNum, key.SmallestKey().UserKey)) 2999 } 3000 if d.cmp(key.LargestKey().UserKey, m.LargestRangeKey.UserKey) > 0 { 3001 panic(errors.Newf("pebble: virtual sstable %s point key %s is not within bounds", m.FileNum, key.LargestKey().UserKey)) 3002 } 3003 } 3004 }