Line data Source code
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
6 :
7 : import (
8 : "bytes"
9 : "context"
10 : "encoding/binary"
11 : "fmt"
12 : "io"
13 : "math"
14 : "os"
15 : "slices"
16 : "sync/atomic"
17 : "time"
18 :
19 : "github.com/cockroachdb/errors"
20 : "github.com/cockroachdb/errors/oserror"
21 : "github.com/cockroachdb/pebble/batchrepr"
22 : "github.com/cockroachdb/pebble/internal/arenaskl"
23 : "github.com/cockroachdb/pebble/internal/base"
24 : "github.com/cockroachdb/pebble/internal/cache"
25 : "github.com/cockroachdb/pebble/internal/constants"
26 : "github.com/cockroachdb/pebble/internal/invariants"
27 : "github.com/cockroachdb/pebble/internal/manifest"
28 : "github.com/cockroachdb/pebble/internal/manual"
29 : "github.com/cockroachdb/pebble/objstorage"
30 : "github.com/cockroachdb/pebble/objstorage/objstorageprovider"
31 : "github.com/cockroachdb/pebble/objstorage/remote"
32 : "github.com/cockroachdb/pebble/record"
33 : "github.com/cockroachdb/pebble/sstable"
34 : "github.com/cockroachdb/pebble/vfs"
35 : "github.com/cockroachdb/pebble/wal"
36 : "github.com/prometheus/client_golang/prometheus"
37 : )
38 :
39 : const (
40 : initialMemTableSize = 256 << 10 // 256 KB
41 :
42 : // The max batch size is limited by the uint32 offsets stored in
43 : // internal/batchskl.node, DeferredBatchOp, and flushableBatchEntry.
44 : //
45 : // We limit the size to MaxUint32 (just short of 4GB) so that the exclusive
46 : // end of an allocation fits in uint32.
47 : //
48 : // On 32-bit systems, slices are naturally limited to MaxInt (just short of
49 : // 2GB).
50 : maxBatchSize = constants.MaxUint32OrInt
51 :
52 : // The max memtable size is limited by the uint32 offsets stored in
53 : // internal/arenaskl.node, DeferredBatchOp, and flushableBatchEntry.
54 : //
55 : // We limit the size to MaxUint32 (just short of 4GB) so that the exclusive
56 : // end of an allocation fits in uint32.
57 : //
58 : // On 32-bit systems, slices are naturally limited to MaxInt (just short of
59 : // 2GB).
60 : maxMemTableSize = constants.MaxUint32OrInt
61 : )
62 :
63 : // TableCacheSize can be used to determine the table
64 : // cache size for a single db, given the maximum open
65 : // files which can be used by a table cache which is
66 : // only used by a single db.
67 1 : func TableCacheSize(maxOpenFiles int) int {
68 1 : tableCacheSize := maxOpenFiles - numNonTableCacheFiles
69 1 : if tableCacheSize < minTableCacheSize {
70 0 : tableCacheSize = minTableCacheSize
71 0 : }
72 1 : return tableCacheSize
73 : }
74 :
75 : // Open opens a DB whose files live in the given directory.
76 1 : func Open(dirname string, opts *Options) (db *DB, err error) {
77 1 : // Make a copy of the options so that we don't mutate the passed in options.
78 1 : opts = opts.Clone()
79 1 : opts = opts.EnsureDefaults()
80 1 : if err := opts.Validate(); err != nil {
81 0 : return nil, err
82 0 : }
83 1 : if opts.LoggerAndTracer == nil {
84 1 : opts.LoggerAndTracer = &base.LoggerWithNoopTracer{Logger: opts.Logger}
85 1 : } else {
86 1 : opts.Logger = opts.LoggerAndTracer
87 1 : }
88 :
89 : // In all error cases, we return db = nil; this is used by various
90 : // deferred cleanups.
91 :
92 : // Open the database and WAL directories first.
93 1 : walDirname, dataDir, err := prepareAndOpenDirs(dirname, opts)
94 1 : if err != nil {
95 1 : return nil, errors.Wrapf(err, "error opening database at %q", dirname)
96 1 : }
97 1 : defer func() {
98 1 : if db == nil {
99 1 : dataDir.Close()
100 1 : }
101 : }()
102 :
103 : // Lock the database directory.
104 1 : var fileLock *Lock
105 1 : if opts.Lock != nil {
106 1 : // The caller already acquired the database lock. Ensure that the
107 1 : // directory matches.
108 1 : if err := opts.Lock.pathMatches(dirname); err != nil {
109 0 : return nil, err
110 0 : }
111 1 : if err := opts.Lock.refForOpen(); err != nil {
112 1 : return nil, err
113 1 : }
114 1 : fileLock = opts.Lock
115 1 : } else {
116 1 : fileLock, err = LockDirectory(dirname, opts.FS)
117 1 : if err != nil {
118 1 : return nil, err
119 1 : }
120 : }
121 1 : defer func() {
122 1 : if db == nil {
123 1 : fileLock.Close()
124 1 : }
125 : }()
126 :
127 : // Establish the format major version.
128 1 : formatVersion, formatVersionMarker, err := lookupFormatMajorVersion(opts.FS, dirname)
129 1 : if err != nil {
130 1 : return nil, err
131 1 : }
132 1 : defer func() {
133 1 : if db == nil {
134 1 : formatVersionMarker.Close()
135 1 : }
136 : }()
137 :
138 1 : noFormatVersionMarker := formatVersion == FormatDefault
139 1 : if noFormatVersionMarker {
140 1 : // We will initialize the store at the minimum possible format, then upgrade
141 1 : // the format to the desired one. This helps test the format upgrade code.
142 1 : formatVersion = FormatMinSupported
143 1 : if opts.Experimental.CreateOnShared != remote.CreateOnSharedNone {
144 1 : formatVersion = FormatMinForSharedObjects
145 1 : }
146 : // There is no format version marker file. There are three cases:
147 : // - we are trying to open an existing store that was created at
148 : // FormatMostCompatible (the only one without a version marker file)
149 : // - we are creating a new store;
150 : // - we are retrying a failed creation.
151 : //
152 : // To error in the first case, we set ErrorIfNotPristine.
153 1 : opts.ErrorIfNotPristine = true
154 1 : defer func() {
155 1 : if err != nil && errors.Is(err, ErrDBNotPristine) {
156 0 : // We must be trying to open an existing store at FormatMostCompatible.
157 0 : // Correct the error in this case -we
158 0 : err = errors.Newf(
159 0 : "pebble: database %q written in format major version 1 which is no longer supported",
160 0 : dirname)
161 0 : }
162 : }()
163 1 : } else {
164 1 : if opts.Experimental.CreateOnShared != remote.CreateOnSharedNone && formatVersion < FormatMinForSharedObjects {
165 0 : return nil, errors.Newf(
166 0 : "pebble: database %q configured with shared objects but written in too old format major version %d",
167 0 : formatVersion)
168 0 : }
169 : }
170 :
171 : // Find the currently active manifest, if there is one.
172 1 : manifestMarker, manifestFileNum, manifestExists, err := findCurrentManifest(opts.FS, dirname)
173 1 : if err != nil {
174 1 : return nil, errors.Wrapf(err, "pebble: database %q", dirname)
175 1 : }
176 1 : defer func() {
177 1 : if db == nil {
178 1 : manifestMarker.Close()
179 1 : }
180 : }()
181 :
182 : // Atomic markers may leave behind obsolete files if there's a crash
183 : // mid-update. Clean these up if we're not in read-only mode.
184 1 : if !opts.ReadOnly {
185 1 : if err := formatVersionMarker.RemoveObsolete(); err != nil {
186 0 : return nil, err
187 0 : }
188 1 : if err := manifestMarker.RemoveObsolete(); err != nil {
189 0 : return nil, err
190 0 : }
191 : }
192 :
193 1 : if opts.Cache == nil {
194 1 : opts.Cache = cache.New(cacheDefaultSize)
195 1 : } else {
196 1 : opts.Cache.Ref()
197 1 : }
198 :
199 1 : d := &DB{
200 1 : cacheID: opts.Cache.NewID(),
201 1 : dirname: dirname,
202 1 : opts: opts,
203 1 : cmp: opts.Comparer.Compare,
204 1 : equal: opts.Comparer.Equal,
205 1 : merge: opts.Merger.Merge,
206 1 : split: opts.Comparer.Split,
207 1 : abbreviatedKey: opts.Comparer.AbbreviatedKey,
208 1 : largeBatchThreshold: (opts.MemTableSize - uint64(memTableEmptySize)) / 2,
209 1 : fileLock: fileLock,
210 1 : dataDir: dataDir,
211 1 : closed: new(atomic.Value),
212 1 : closedCh: make(chan struct{}),
213 1 : }
214 1 : d.mu.versions = &versionSet{}
215 1 : d.diskAvailBytes.Store(math.MaxUint64)
216 1 :
217 1 : defer func() {
218 1 : // If an error or panic occurs during open, attempt to release the manually
219 1 : // allocated memory resources. Note that rather than look for an error, we
220 1 : // look for the return of a nil DB pointer.
221 1 : if r := recover(); db == nil {
222 1 : // Release our references to the Cache. Note that both the DB, and
223 1 : // tableCache have a reference. When we release the reference to
224 1 : // the tableCache, and if there are no other references to
225 1 : // the tableCache, then the tableCache will also release its
226 1 : // reference to the cache.
227 1 : opts.Cache.Unref()
228 1 :
229 1 : if d.tableCache != nil {
230 1 : _ = d.tableCache.close()
231 1 : }
232 :
233 1 : for _, mem := range d.mu.mem.queue {
234 1 : switch t := mem.flushable.(type) {
235 1 : case *memTable:
236 1 : manual.Free(t.arenaBuf)
237 1 : t.arenaBuf = nil
238 : }
239 : }
240 1 : if d.cleanupManager != nil {
241 1 : d.cleanupManager.Close()
242 1 : }
243 1 : if d.objProvider != nil {
244 1 : d.objProvider.Close()
245 1 : }
246 1 : if r != nil {
247 1 : panic(r)
248 : }
249 : }
250 : }()
251 :
252 1 : d.commit = newCommitPipeline(commitEnv{
253 1 : logSeqNum: &d.mu.versions.logSeqNum,
254 1 : visibleSeqNum: &d.mu.versions.visibleSeqNum,
255 1 : apply: d.commitApply,
256 1 : write: d.commitWrite,
257 1 : })
258 1 : d.mu.nextJobID = 1
259 1 : d.mu.mem.nextSize = opts.MemTableSize
260 1 : if d.mu.mem.nextSize > initialMemTableSize {
261 1 : d.mu.mem.nextSize = initialMemTableSize
262 1 : }
263 1 : d.mu.compact.cond.L = &d.mu.Mutex
264 1 : d.mu.compact.inProgress = make(map[*compaction]struct{})
265 1 : d.mu.compact.noOngoingFlushStartTime = time.Now()
266 1 : d.mu.snapshots.init()
267 1 : // logSeqNum is the next sequence number that will be assigned.
268 1 : // Start assigning sequence numbers from base.SeqNumStart to leave
269 1 : // room for reserved sequence numbers (see comments around
270 1 : // SeqNumStart).
271 1 : d.mu.versions.logSeqNum.Store(base.SeqNumStart)
272 1 : d.mu.formatVers.vers.Store(uint64(formatVersion))
273 1 : d.mu.formatVers.marker = formatVersionMarker
274 1 :
275 1 : d.timeNow = time.Now
276 1 : d.openedAt = d.timeNow()
277 1 :
278 1 : d.mu.Lock()
279 1 : defer d.mu.Unlock()
280 1 :
281 1 : jobID := d.mu.nextJobID
282 1 : d.mu.nextJobID++
283 1 :
284 1 : if !manifestExists {
285 1 : // DB does not exist.
286 1 : if d.opts.ErrorIfNotExists || d.opts.ReadOnly {
287 1 : return nil, errors.Wrapf(ErrDBDoesNotExist, "dirname=%q", dirname)
288 1 : }
289 :
290 : // Create the DB.
291 1 : if err := d.mu.versions.create(jobID, dirname, opts, manifestMarker, d.FormatMajorVersion, &d.mu.Mutex); err != nil {
292 1 : return nil, err
293 1 : }
294 1 : } else {
295 1 : if opts.ErrorIfExists {
296 1 : return nil, errors.Wrapf(ErrDBAlreadyExists, "dirname=%q", dirname)
297 1 : }
298 : // Load the version set.
299 1 : if err := d.mu.versions.load(dirname, opts, manifestFileNum, manifestMarker, d.FormatMajorVersion, &d.mu.Mutex); err != nil {
300 1 : return nil, err
301 1 : }
302 1 : if opts.ErrorIfNotPristine {
303 1 : liveFileNums := make(map[base.DiskFileNum]struct{})
304 1 : d.mu.versions.addLiveFileNums(liveFileNums)
305 1 : if len(liveFileNums) != 0 {
306 1 : return nil, errors.Wrapf(ErrDBNotPristine, "dirname=%q", dirname)
307 1 : }
308 : }
309 : }
310 :
311 : // In read-only mode, we replay directly into the mutable memtable but never
312 : // flush it. We need to delay creation of the memtable until we know the
313 : // sequence number of the first batch that will be inserted.
314 1 : if !d.opts.ReadOnly {
315 1 : var entry *flushableEntry
316 1 : d.mu.mem.mutable, entry = d.newMemTable(0 /* logNum */, d.mu.versions.logSeqNum.Load())
317 1 : d.mu.mem.queue = append(d.mu.mem.queue, entry)
318 1 : }
319 :
320 1 : d.mu.log.metrics.fsyncLatency = prometheus.NewHistogram(prometheus.HistogramOpts{
321 1 : Buckets: FsyncLatencyBuckets,
322 1 : })
323 1 : walOpts := wal.Options{
324 1 : Primary: wal.Dir{FS: opts.FS, Dirname: walDirname},
325 1 : Secondary: wal.Dir{},
326 1 : MinUnflushedWALNum: wal.NumWAL(d.mu.versions.minUnflushedLogNum),
327 1 : MaxNumRecyclableLogs: opts.MemTableStopWritesThreshold + 1,
328 1 : NoSyncOnClose: opts.NoSyncOnClose,
329 1 : BytesPerSync: opts.WALBytesPerSync,
330 1 : PreallocateSize: d.walPreallocateSize,
331 1 : MinSyncInterval: opts.WALMinSyncInterval,
332 1 : FsyncLatency: d.mu.log.metrics.fsyncLatency,
333 1 : QueueSemChan: d.commit.logSyncQSem,
334 1 : Logger: opts.Logger,
335 1 : EventListener: walEventListenerAdaptor{l: opts.EventListener},
336 1 : }
337 1 : if opts.WALFailover != nil {
338 1 : walOpts.Secondary = opts.WALFailover.Secondary
339 1 : walOpts.FailoverOptions = opts.WALFailover.FailoverOptions
340 1 : }
341 1 : walDirs := append(walOpts.Dirs(), opts.WALRecoveryDirs...)
342 1 : wals, err := wal.Scan(walDirs...)
343 1 : if err != nil {
344 1 : return nil, err
345 1 : }
346 1 : walManager, err := wal.Init(walOpts, wals)
347 1 : if err != nil {
348 1 : return nil, err
349 1 : }
350 1 : defer func() {
351 1 : if db == nil {
352 1 : walManager.Close()
353 1 : }
354 : }()
355 :
356 1 : d.mu.log.manager = walManager
357 1 :
358 1 : // List the objects. This also happens to include WAL log files, if they are
359 1 : // in the same dir, but we will ignore those below. The provider is also
360 1 : // given this list, but it ignores non sstable files.
361 1 : ls, err := opts.FS.List(d.dirname)
362 1 : if err != nil {
363 1 : return nil, err
364 1 : }
365 1 : providerSettings := objstorageprovider.Settings{
366 1 : Logger: opts.Logger,
367 1 : FS: opts.FS,
368 1 : FSDirName: dirname,
369 1 : FSDirInitialListing: ls,
370 1 : FSCleaner: opts.Cleaner,
371 1 : NoSyncOnClose: opts.NoSyncOnClose,
372 1 : BytesPerSync: opts.BytesPerSync,
373 1 : }
374 1 : providerSettings.Remote.StorageFactory = opts.Experimental.RemoteStorage
375 1 : providerSettings.Remote.CreateOnShared = opts.Experimental.CreateOnShared
376 1 : providerSettings.Remote.CreateOnSharedLocator = opts.Experimental.CreateOnSharedLocator
377 1 : providerSettings.Remote.CacheSizeBytes = opts.Experimental.SecondaryCacheSizeBytes
378 1 :
379 1 : d.objProvider, err = objstorageprovider.Open(providerSettings)
380 1 : if err != nil {
381 1 : return nil, err
382 1 : }
383 :
384 1 : d.cleanupManager = openCleanupManager(opts, d.objProvider, d.onObsoleteTableDelete, d.getDeletionPacerInfo)
385 1 :
386 1 : if manifestExists {
387 1 : curVersion := d.mu.versions.currentVersion()
388 1 : if err := checkConsistency(curVersion, dirname, d.objProvider); err != nil {
389 0 : return nil, err
390 0 : }
391 : }
392 :
393 1 : tableCacheSize := TableCacheSize(opts.MaxOpenFiles)
394 1 : d.tableCache = newTableCacheContainer(
395 1 : opts.TableCache, d.cacheID, d.objProvider, d.opts, tableCacheSize,
396 1 : &sstable.CategoryStatsCollector{})
397 1 : d.newIters = d.tableCache.newIters
398 1 : d.tableNewRangeKeyIter = tableNewRangeKeyIter(context.TODO(), d.newIters)
399 1 :
400 1 : var previousOptionsFileNum base.DiskFileNum
401 1 : var previousOptionsFilename string
402 1 : for _, filename := range ls {
403 1 : ft, fn, ok := base.ParseFilename(opts.FS, filename)
404 1 : if !ok {
405 1 : continue
406 : }
407 :
408 : // Don't reuse any obsolete file numbers to avoid modifying an
409 : // ingested sstable's original external file.
410 1 : d.mu.versions.markFileNumUsed(fn)
411 1 :
412 1 : switch ft {
413 0 : case fileTypeLog:
414 : // Ignore.
415 1 : case fileTypeOptions:
416 1 : if previousOptionsFileNum < fn {
417 1 : previousOptionsFileNum = fn
418 1 : previousOptionsFilename = filename
419 1 : }
420 1 : case fileTypeTemp, fileTypeOldTemp:
421 1 : if !d.opts.ReadOnly {
422 1 : // Some codepaths write to a temporary file and then
423 1 : // rename it to its final location when complete. A
424 1 : // temp file is leftover if a process exits before the
425 1 : // rename. Remove it.
426 1 : err := opts.FS.Remove(opts.FS.PathJoin(dirname, filename))
427 1 : if err != nil {
428 0 : return nil, err
429 0 : }
430 : }
431 : }
432 : }
433 1 : if n := len(wals); n > 0 {
434 1 : // Don't reuse any obsolete file numbers to avoid modifying an
435 1 : // ingested sstable's original external file.
436 1 : d.mu.versions.markFileNumUsed(base.DiskFileNum(wals[n-1].Num))
437 1 : }
438 :
439 : // Ratchet d.mu.versions.nextFileNum ahead of all known objects in the
440 : // objProvider. This avoids FileNum collisions with obsolete sstables.
441 1 : objects := d.objProvider.List()
442 1 : for _, obj := range objects {
443 1 : d.mu.versions.markFileNumUsed(obj.DiskFileNum)
444 1 : }
445 :
446 : // Validate the most-recent OPTIONS file, if there is one.
447 1 : var strictWALTail bool
448 1 : if previousOptionsFilename != "" {
449 1 : path := opts.FS.PathJoin(dirname, previousOptionsFilename)
450 1 : strictWALTail, err = checkOptions(opts, path)
451 1 : if err != nil {
452 1 : return nil, err
453 1 : }
454 : }
455 :
456 : // Replay any newer log files than the ones named in the manifest.
457 1 : var replayWALs wal.Logs
458 1 : for i, w := range wals {
459 1 : if base.DiskFileNum(w.Num) >= d.mu.versions.minUnflushedLogNum {
460 1 : replayWALs = wals[i:]
461 1 : break
462 : }
463 : }
464 1 : var ve versionEdit
465 1 : var toFlush flushableList
466 1 : for i, lf := range replayWALs {
467 1 : lastWAL := i == len(replayWALs)-1
468 1 : flush, maxSeqNum, err := d.replayWAL(jobID, &ve, lf, strictWALTail && !lastWAL)
469 1 : if err != nil {
470 1 : return nil, err
471 1 : }
472 1 : toFlush = append(toFlush, flush...)
473 1 : if d.mu.versions.logSeqNum.Load() < maxSeqNum {
474 1 : d.mu.versions.logSeqNum.Store(maxSeqNum)
475 1 : }
476 : }
477 1 : d.mu.versions.visibleSeqNum.Store(d.mu.versions.logSeqNum.Load())
478 1 :
479 1 : if !d.opts.ReadOnly {
480 1 : // Create an empty .log file.
481 1 : newLogNum := d.mu.versions.getNextDiskFileNum()
482 1 :
483 1 : // This logic is slightly different than RocksDB's. Specifically, RocksDB
484 1 : // sets MinUnflushedLogNum to max-recovered-log-num + 1. We set it to the
485 1 : // newLogNum. There should be no difference in using either value.
486 1 : ve.MinUnflushedLogNum = newLogNum
487 1 :
488 1 : // Create the manifest with the updated MinUnflushedLogNum before
489 1 : // creating the new log file. If we created the log file first, a
490 1 : // crash before the manifest is synced could leave two WALs with
491 1 : // unclean tails.
492 1 : d.mu.versions.logLock()
493 1 : if err := d.mu.versions.logAndApply(jobID, &ve, newFileMetrics(ve.NewFiles), false /* forceRotation */, func() []compactionInfo {
494 1 : return nil
495 1 : }); err != nil {
496 0 : return nil, err
497 0 : }
498 :
499 1 : for _, entry := range toFlush {
500 1 : entry.readerUnrefLocked(true)
501 1 : }
502 :
503 1 : d.mu.log.writer, err = d.mu.log.manager.Create(wal.NumWAL(newLogNum), jobID)
504 1 : if err != nil {
505 1 : return nil, err
506 1 : }
507 :
508 : // This isn't strictly necessary as we don't use the log number for
509 : // memtables being flushed, only for the next unflushed memtable.
510 1 : d.mu.mem.queue[len(d.mu.mem.queue)-1].logNum = newLogNum
511 : }
512 1 : d.updateReadStateLocked(d.opts.DebugCheck)
513 1 :
514 1 : if !d.opts.ReadOnly {
515 1 : // If the Options specify a format major version higher than the
516 1 : // loaded database's, upgrade it. If this is a new database, this
517 1 : // code path also performs an initial upgrade from the starting
518 1 : // implicit MinSupported version.
519 1 : //
520 1 : // We ratchet the version this far into Open so that migrations have a read
521 1 : // state available. Note that this also results in creating/updating the
522 1 : // format version marker file.
523 1 : if opts.FormatMajorVersion > d.FormatMajorVersion() {
524 1 : if err := d.ratchetFormatMajorVersionLocked(opts.FormatMajorVersion); err != nil {
525 0 : return nil, err
526 0 : }
527 1 : } else if noFormatVersionMarker {
528 1 : // We are creating a new store. Create the format version marker file.
529 1 : if err := d.writeFormatVersionMarker(d.FormatMajorVersion()); err != nil {
530 1 : return nil, err
531 1 : }
532 : }
533 :
534 : // Write the current options to disk.
535 1 : d.optionsFileNum = d.mu.versions.getNextDiskFileNum()
536 1 : tmpPath := base.MakeFilepath(opts.FS, dirname, fileTypeTemp, d.optionsFileNum)
537 1 : optionsPath := base.MakeFilepath(opts.FS, dirname, fileTypeOptions, d.optionsFileNum)
538 1 :
539 1 : // Write them to a temporary file first, in case we crash before
540 1 : // we're done. A corrupt options file prevents opening the
541 1 : // database.
542 1 : optionsFile, err := opts.FS.Create(tmpPath)
543 1 : if err != nil {
544 1 : return nil, err
545 1 : }
546 1 : serializedOpts := []byte(opts.String())
547 1 : if _, err := optionsFile.Write(serializedOpts); err != nil {
548 1 : return nil, errors.CombineErrors(err, optionsFile.Close())
549 1 : }
550 1 : d.optionsFileSize = uint64(len(serializedOpts))
551 1 : if err := optionsFile.Sync(); err != nil {
552 1 : return nil, errors.CombineErrors(err, optionsFile.Close())
553 1 : }
554 1 : if err := optionsFile.Close(); err != nil {
555 0 : return nil, err
556 0 : }
557 : // Atomically rename to the OPTIONS-XXXXXX path. This rename is
558 : // guaranteed to be atomic because the destination path does not
559 : // exist.
560 1 : if err := opts.FS.Rename(tmpPath, optionsPath); err != nil {
561 1 : return nil, err
562 1 : }
563 1 : if err := d.dataDir.Sync(); err != nil {
564 1 : return nil, err
565 1 : }
566 : }
567 :
568 1 : if !d.opts.ReadOnly {
569 1 : d.scanObsoleteFiles(ls)
570 1 : d.deleteObsoleteFiles(jobID)
571 1 : }
572 : // Else, nothing is obsolete.
573 :
574 1 : d.mu.tableStats.cond.L = &d.mu.Mutex
575 1 : d.mu.tableValidation.cond.L = &d.mu.Mutex
576 1 : if !d.opts.ReadOnly {
577 1 : d.maybeCollectTableStatsLocked()
578 1 : }
579 1 : d.calculateDiskAvailableBytes()
580 1 :
581 1 : d.maybeScheduleFlush()
582 1 : d.maybeScheduleCompaction()
583 1 :
584 1 : // Note: this is a no-op if invariants are disabled or race is enabled.
585 1 : //
586 1 : // Setting a finalizer on *DB causes *DB to never be reclaimed and the
587 1 : // finalizer to never be run. The problem is due to this limitation of
588 1 : // finalizers mention in the SetFinalizer docs:
589 1 : //
590 1 : // If a cyclic structure includes a block with a finalizer, that cycle is
591 1 : // not guaranteed to be garbage collected and the finalizer is not
592 1 : // guaranteed to run, because there is no ordering that respects the
593 1 : // dependencies.
594 1 : //
595 1 : // DB has cycles with several of its internal structures: readState,
596 1 : // newIters, tableCache, versions, etc. Each of this individually cause a
597 1 : // cycle and prevent the finalizer from being run. But we can workaround this
598 1 : // finializer limitation by setting a finalizer on another object that is
599 1 : // tied to the lifetime of DB: the DB.closed atomic.Value.
600 1 : dPtr := fmt.Sprintf("%p", d)
601 1 : invariants.SetFinalizer(d.closed, func(obj interface{}) {
602 0 : v := obj.(*atomic.Value)
603 0 : if err := v.Load(); err == nil {
604 0 : fmt.Fprintf(os.Stderr, "%s: unreferenced DB not closed\n", dPtr)
605 0 : os.Exit(1)
606 0 : }
607 : })
608 :
609 1 : return d, nil
610 : }
611 :
612 : // prepareAndOpenDirs opens the directories for the store (and creates them if
613 : // necessary).
614 : //
615 : // Returns an error if ReadOnly is set and the directories don't exist.
616 : func prepareAndOpenDirs(
617 : dirname string, opts *Options,
618 1 : ) (walDirname string, dataDir vfs.File, err error) {
619 1 : walDirname = opts.WALDir
620 1 : if opts.WALDir == "" {
621 1 : walDirname = dirname
622 1 : }
623 :
624 : // Create directories if needed.
625 1 : if !opts.ReadOnly {
626 1 : f, err := mkdirAllAndSyncParents(opts.FS, dirname)
627 1 : if err != nil {
628 1 : return "", nil, err
629 1 : }
630 1 : f.Close()
631 1 : if walDirname != dirname {
632 1 : f, err := mkdirAllAndSyncParents(opts.FS, walDirname)
633 1 : if err != nil {
634 0 : return "", nil, err
635 0 : }
636 1 : f.Close()
637 : }
638 1 : if opts.WALFailover != nil {
639 1 : secondary := opts.WALFailover.Secondary
640 1 : f, err := mkdirAllAndSyncParents(secondary.FS, secondary.Dirname)
641 1 : if err != nil {
642 0 : return "", nil, err
643 0 : }
644 1 : f.Close()
645 : }
646 : }
647 :
648 1 : dataDir, err = opts.FS.OpenDir(dirname)
649 1 : if err != nil {
650 1 : if opts.ReadOnly && oserror.IsNotExist(err) {
651 1 : return "", nil, errors.Errorf("pebble: database %q does not exist", dirname)
652 1 : }
653 1 : return "", nil, err
654 : }
655 1 : if opts.ReadOnly && walDirname != dirname {
656 1 : // Check that the wal dir exists.
657 1 : walDir, err := opts.FS.OpenDir(walDirname)
658 1 : if err != nil {
659 1 : dataDir.Close()
660 1 : return "", nil, err
661 1 : }
662 1 : walDir.Close()
663 : }
664 :
665 1 : return walDirname, dataDir, nil
666 : }
667 :
668 : // GetVersion returns the engine version string from the latest options
669 : // file present in dir. Used to check what Pebble or RocksDB version was last
670 : // used to write to the database stored in this directory. An empty string is
671 : // returned if no valid OPTIONS file with a version key was found.
672 1 : func GetVersion(dir string, fs vfs.FS) (string, error) {
673 1 : ls, err := fs.List(dir)
674 1 : if err != nil {
675 0 : return "", err
676 0 : }
677 1 : var version string
678 1 : lastOptionsSeen := base.DiskFileNum(0)
679 1 : for _, filename := range ls {
680 1 : ft, fn, ok := base.ParseFilename(fs, filename)
681 1 : if !ok {
682 1 : continue
683 : }
684 1 : switch ft {
685 1 : case fileTypeOptions:
686 1 : // If this file has a higher number than the last options file
687 1 : // processed, reset version. This is because rocksdb often
688 1 : // writes multiple options files without deleting previous ones.
689 1 : // Otherwise, skip parsing this options file.
690 1 : if fn > lastOptionsSeen {
691 1 : version = ""
692 1 : lastOptionsSeen = fn
693 1 : } else {
694 1 : continue
695 : }
696 1 : f, err := fs.Open(fs.PathJoin(dir, filename))
697 1 : if err != nil {
698 0 : return "", err
699 0 : }
700 1 : data, err := io.ReadAll(f)
701 1 : f.Close()
702 1 :
703 1 : if err != nil {
704 0 : return "", err
705 0 : }
706 1 : err = parseOptions(string(data), func(section, key, value string) error {
707 1 : switch {
708 1 : case section == "Version":
709 1 : switch key {
710 1 : case "pebble_version":
711 1 : version = value
712 1 : case "rocksdb_version":
713 1 : version = fmt.Sprintf("rocksdb v%s", value)
714 : }
715 : }
716 1 : return nil
717 : })
718 1 : if err != nil {
719 0 : return "", err
720 0 : }
721 : }
722 : }
723 1 : return version, nil
724 : }
725 :
726 : // replayWAL replays the edits in the specified WAL. If the DB is in read
727 : // only mode, then the WALs are replayed into memtables and not flushed. If
728 : // the DB is not in read only mode, then the contents of the WAL are
729 : // guaranteed to be flushed. Note that this flushing is very important for
730 : // guaranteeing durability: the application may have had a number of pending
731 : // fsyncs to the WAL before the process crashed, and those fsyncs may not have
732 : // happened but the corresponding data may now be readable from the WAL (while
733 : // sitting in write-back caches in the kernel or the storage device). By
734 : // reading the WAL (including the non-fsynced data) and then flushing all
735 : // these changes (flush does fsyncs), we are able to guarantee that the
736 : // initial state of the DB is durable.
737 : //
738 : // The toFlush return value is a list of flushables associated with the WAL
739 : // being replayed which will be flushed. Once the version edit has been applied
740 : // to the manifest, it is up to the caller of replayWAL to unreference the
741 : // toFlush flushables returned by replayWAL.
742 : //
743 : // d.mu must be held when calling this, but the mutex may be dropped and
744 : // re-acquired during the course of this method.
745 : func (d *DB) replayWAL(
746 : jobID int, ve *versionEdit, ll wal.LogicalLog, strictWALTail bool,
747 1 : ) (toFlush flushableList, maxSeqNum uint64, err error) {
748 1 : rr := ll.OpenForRead()
749 1 : defer rr.Close()
750 1 : var (
751 1 : b Batch
752 1 : buf bytes.Buffer
753 1 : mem *memTable
754 1 : entry *flushableEntry
755 1 : offset int64 // byte offset in rr
756 1 : lastFlushOffset int64
757 1 : keysReplayed int64 // number of keys replayed
758 1 : batchesReplayed int64 // number of batches replayed
759 1 : )
760 1 :
761 1 : // TODO(jackson): This function is interspersed with panics, in addition to
762 1 : // corruption error propagation. Audit them to ensure we're truly only
763 1 : // panicking where the error points to Pebble bug and not user or
764 1 : // hardware-induced corruption.
765 1 :
766 1 : if d.opts.ReadOnly {
767 1 : // In read-only mode, we replay directly into the mutable memtable which will
768 1 : // never be flushed.
769 1 : mem = d.mu.mem.mutable
770 1 : if mem != nil {
771 1 : entry = d.mu.mem.queue[len(d.mu.mem.queue)-1]
772 1 : }
773 : }
774 :
775 : // Flushes the current memtable, if not nil.
776 1 : flushMem := func() {
777 1 : if mem == nil {
778 1 : return
779 1 : }
780 1 : var logSize uint64
781 1 : if offset >= lastFlushOffset {
782 1 : logSize = uint64(offset - lastFlushOffset)
783 1 : }
784 : // Else, this was the initial memtable in the read-only case which must have
785 : // been empty, but we need to flush it since we don't want to add to it later.
786 1 : lastFlushOffset = offset
787 1 : entry.logSize = logSize
788 1 : if !d.opts.ReadOnly {
789 1 : toFlush = append(toFlush, entry)
790 1 : }
791 1 : mem, entry = nil, nil
792 : }
793 : // Creates a new memtable if there is no current memtable.
794 1 : ensureMem := func(seqNum uint64) {
795 1 : if mem != nil {
796 1 : return
797 1 : }
798 1 : mem, entry = d.newMemTable(base.DiskFileNum(ll.Num), seqNum)
799 1 : if d.opts.ReadOnly {
800 1 : d.mu.mem.mutable = mem
801 1 : d.mu.mem.queue = append(d.mu.mem.queue, entry)
802 1 : }
803 : }
804 :
805 : // updateVE is used to update ve with information about new files created
806 : // during the flush of any flushable not of type ingestedFlushable. For the
807 : // flushable of type ingestedFlushable we use custom handling below.
808 1 : updateVE := func() error {
809 1 : // TODO(bananabrick): See if we can use the actual base level here,
810 1 : // instead of using 1.
811 1 : c, err := newFlush(d.opts, d.mu.versions.currentVersion(),
812 1 : 1 /* base level */, toFlush, d.timeNow())
813 1 : if err != nil {
814 0 : return err
815 0 : }
816 1 : newVE, _, _, err := d.runCompaction(jobID, c)
817 1 : if err != nil {
818 0 : return errors.Wrapf(err, "running compaction during WAL replay")
819 0 : }
820 1 : ve.NewFiles = append(ve.NewFiles, newVE.NewFiles...)
821 1 : return nil
822 : }
823 1 : defer func() {
824 1 : if err != nil {
825 1 : err = errors.WithDetailf(err, "replaying wal %d, offset %d", ll.Num, offset)
826 1 : }
827 : }()
828 :
829 1 : for {
830 1 : r, offset, err := rr.NextRecord()
831 1 : if err == nil {
832 1 : _, err = io.Copy(&buf, r)
833 1 : }
834 1 : if err != nil {
835 1 : // It is common to encounter a zeroed or invalid chunk due to WAL
836 1 : // preallocation and WAL recycling. We need to distinguish these
837 1 : // errors from EOF in order to recognize that the record was
838 1 : // truncated and to avoid replaying subsequent WALs, but want
839 1 : // to otherwise treat them like EOF.
840 1 : if err == io.EOF {
841 1 : break
842 1 : } else if record.IsInvalidRecord(err) && !strictWALTail {
843 1 : break
844 : }
845 1 : return nil, 0, errors.Wrap(err, "pebble: error when replaying WAL")
846 : }
847 :
848 1 : if buf.Len() < batchrepr.HeaderLen {
849 0 : return nil, 0, base.CorruptionErrorf("pebble: corrupt wal %s (offset %s)",
850 0 : errors.Safe(base.DiskFileNum(ll.Num)), offset)
851 0 : }
852 :
853 1 : if d.opts.ErrorIfNotPristine {
854 1 : return nil, 0, errors.WithDetailf(ErrDBNotPristine, "location: %q", d.dirname)
855 1 : }
856 :
857 : // Specify Batch.db so that Batch.SetRepr will compute Batch.memTableSize
858 : // which is used below.
859 1 : b = Batch{}
860 1 : b.db = d
861 1 : b.SetRepr(buf.Bytes())
862 1 : seqNum := b.SeqNum()
863 1 : maxSeqNum = seqNum + uint64(b.Count())
864 1 : keysReplayed += int64(b.Count())
865 1 : batchesReplayed++
866 1 : {
867 1 : br := b.Reader()
868 1 : if kind, encodedFileNum, _, ok, err := br.Next(); err != nil {
869 0 : return nil, 0, err
870 1 : } else if ok && kind == InternalKeyKindIngestSST {
871 1 : fileNums := make([]base.DiskFileNum, 0, b.Count())
872 1 : addFileNum := func(encodedFileNum []byte) {
873 1 : fileNum, n := binary.Uvarint(encodedFileNum)
874 1 : if n <= 0 {
875 0 : panic("pebble: ingest sstable file num is invalid.")
876 : }
877 1 : fileNums = append(fileNums, base.DiskFileNum(fileNum))
878 : }
879 1 : addFileNum(encodedFileNum)
880 1 :
881 1 : for i := 1; i < int(b.Count()); i++ {
882 1 : kind, encodedFileNum, _, ok, err := br.Next()
883 1 : if err != nil {
884 0 : return nil, 0, err
885 0 : }
886 1 : if kind != InternalKeyKindIngestSST {
887 0 : panic("pebble: invalid batch key kind.")
888 : }
889 1 : if !ok {
890 0 : panic("pebble: invalid batch count.")
891 : }
892 1 : addFileNum(encodedFileNum)
893 : }
894 :
895 1 : if _, _, _, ok, err := br.Next(); err != nil {
896 0 : return nil, 0, err
897 1 : } else if ok {
898 0 : panic("pebble: invalid number of entries in batch.")
899 : }
900 :
901 1 : meta := make([]*fileMetadata, len(fileNums))
902 1 : for i, n := range fileNums {
903 1 : var readable objstorage.Readable
904 1 : objMeta, err := d.objProvider.Lookup(fileTypeTable, n)
905 1 : if err != nil {
906 0 : return nil, 0, errors.Wrap(err, "pebble: error when looking up ingested SSTs")
907 0 : }
908 1 : if objMeta.IsRemote() {
909 0 : readable, err = d.objProvider.OpenForReading(context.TODO(), fileTypeTable, n, objstorage.OpenOptions{MustExist: true})
910 0 : if err != nil {
911 0 : return nil, 0, errors.Wrap(err, "pebble: error when opening flushable ingest files")
912 0 : }
913 1 : } else {
914 1 : path := base.MakeFilepath(d.opts.FS, d.dirname, fileTypeTable, n)
915 1 : f, err := d.opts.FS.Open(path)
916 1 : if err != nil {
917 0 : return nil, 0, err
918 0 : }
919 :
920 1 : readable, err = sstable.NewSimpleReadable(f)
921 1 : if err != nil {
922 0 : return nil, 0, err
923 0 : }
924 : }
925 : // NB: ingestLoad1 will close readable.
926 1 : meta[i], err = ingestLoad1(d.opts, d.FormatMajorVersion(), readable, d.cacheID, base.PhysicalTableFileNum(n))
927 1 : if err != nil {
928 0 : return nil, 0, errors.Wrap(err, "pebble: error when loading flushable ingest files")
929 0 : }
930 : }
931 :
932 1 : if uint32(len(meta)) != b.Count() {
933 0 : panic("pebble: couldn't load all files in WAL entry.")
934 : }
935 :
936 1 : entry, err = d.newIngestedFlushableEntry(
937 1 : meta, seqNum, base.DiskFileNum(ll.Num),
938 1 : )
939 1 : if err != nil {
940 0 : return nil, 0, err
941 0 : }
942 :
943 1 : if d.opts.ReadOnly {
944 1 : d.mu.mem.queue = append(d.mu.mem.queue, entry)
945 1 : // We added the IngestSST flushable to the queue. But there
946 1 : // must be at least one WAL entry waiting to be replayed. We
947 1 : // have to ensure this newer WAL entry isn't replayed into
948 1 : // the current value of d.mu.mem.mutable because the current
949 1 : // mutable memtable exists before this flushable entry in
950 1 : // the memtable queue. To ensure this, we just need to unset
951 1 : // d.mu.mem.mutable. When a newer WAL is replayed, we will
952 1 : // set d.mu.mem.mutable to a newer value.
953 1 : d.mu.mem.mutable = nil
954 1 : } else {
955 1 : toFlush = append(toFlush, entry)
956 1 : // During WAL replay, the lsm only has L0, hence, the
957 1 : // baseLevel is 1. For the sake of simplicity, we place the
958 1 : // ingested files in L0 here, instead of finding their
959 1 : // target levels. This is a simplification for the sake of
960 1 : // simpler code. It is expected that WAL replay should be
961 1 : // rare, and that flushables of type ingestedFlushable
962 1 : // should also be rare. So, placing the ingested files in L0
963 1 : // is alright.
964 1 : //
965 1 : // TODO(bananabrick): Maybe refactor this function to allow
966 1 : // us to easily place ingested files in levels as low as
967 1 : // possible during WAL replay. It would require breaking up
968 1 : // the application of ve to the manifest into chunks and is
969 1 : // not pretty w/o a refactor to this function and how it's
970 1 : // used.
971 1 : c, err := newFlush(
972 1 : d.opts, d.mu.versions.currentVersion(),
973 1 : 1, /* base level */
974 1 : []*flushableEntry{entry},
975 1 : d.timeNow(),
976 1 : )
977 1 : if err != nil {
978 0 : return nil, 0, err
979 0 : }
980 1 : for _, file := range c.flushing[0].flushable.(*ingestedFlushable).files {
981 1 : ve.NewFiles = append(ve.NewFiles, newFileEntry{Level: 0, Meta: file.FileMetadata})
982 1 : }
983 : }
984 1 : return toFlush, maxSeqNum, nil
985 : }
986 : }
987 :
988 1 : if b.memTableSize >= uint64(d.largeBatchThreshold) {
989 1 : flushMem()
990 1 : // Make a copy of the data slice since it is currently owned by buf and will
991 1 : // be reused in the next iteration.
992 1 : b.data = slices.Clone(b.data)
993 1 : b.flushable, err = newFlushableBatch(&b, d.opts.Comparer)
994 1 : if err != nil {
995 0 : return nil, 0, err
996 0 : }
997 1 : entry := d.newFlushableEntry(b.flushable, base.DiskFileNum(ll.Num), b.SeqNum())
998 1 : // Disable memory accounting by adding a reader ref that will never be
999 1 : // removed.
1000 1 : entry.readerRefs.Add(1)
1001 1 : if d.opts.ReadOnly {
1002 1 : d.mu.mem.queue = append(d.mu.mem.queue, entry)
1003 1 : // We added the flushable batch to the flushable to the queue.
1004 1 : // But there must be at least one WAL entry waiting to be
1005 1 : // replayed. We have to ensure this newer WAL entry isn't
1006 1 : // replayed into the current value of d.mu.mem.mutable because
1007 1 : // the current mutable memtable exists before this flushable
1008 1 : // entry in the memtable queue. To ensure this, we just need to
1009 1 : // unset d.mu.mem.mutable. When a newer WAL is replayed, we will
1010 1 : // set d.mu.mem.mutable to a newer value.
1011 1 : d.mu.mem.mutable = nil
1012 1 : } else {
1013 1 : toFlush = append(toFlush, entry)
1014 1 : }
1015 1 : } else {
1016 1 : ensureMem(seqNum)
1017 1 : if err = mem.prepare(&b); err != nil && err != arenaskl.ErrArenaFull {
1018 0 : return nil, 0, err
1019 0 : }
1020 : // We loop since DB.newMemTable() slowly grows the size of allocated memtables, so the
1021 : // batch may not initially fit, but will eventually fit (since it is smaller than
1022 : // largeBatchThreshold).
1023 1 : for err == arenaskl.ErrArenaFull {
1024 1 : flushMem()
1025 1 : ensureMem(seqNum)
1026 1 : err = mem.prepare(&b)
1027 1 : if err != nil && err != arenaskl.ErrArenaFull {
1028 0 : return nil, 0, err
1029 0 : }
1030 : }
1031 1 : if err = mem.apply(&b, seqNum); err != nil {
1032 0 : return nil, 0, err
1033 0 : }
1034 1 : mem.writerUnref()
1035 : }
1036 1 : buf.Reset()
1037 : }
1038 :
1039 1 : d.opts.Logger.Infof("[JOB %d] WAL %s stopped reading at offset: %d; replayed %d keys in %d batches",
1040 1 : jobID, base.DiskFileNum(ll.Num).String(), offset, keysReplayed, batchesReplayed)
1041 1 : flushMem()
1042 1 :
1043 1 : // mem is nil here.
1044 1 : if !d.opts.ReadOnly && batchesReplayed > 0 {
1045 1 : err = updateVE()
1046 1 : if err != nil {
1047 0 : return nil, 0, err
1048 0 : }
1049 : }
1050 1 : return toFlush, maxSeqNum, err
1051 : }
1052 :
1053 1 : func checkOptions(opts *Options, path string) (strictWALTail bool, err error) {
1054 1 : f, err := opts.FS.Open(path)
1055 1 : if err != nil {
1056 0 : return false, err
1057 0 : }
1058 1 : defer f.Close()
1059 1 :
1060 1 : data, err := io.ReadAll(f)
1061 1 : if err != nil {
1062 0 : return false, err
1063 0 : }
1064 1 : return opts.checkOptions(string(data))
1065 : }
1066 :
1067 : // DBDesc briefly describes high-level state about a database.
1068 : type DBDesc struct {
1069 : // Exists is true if an existing database was found.
1070 : Exists bool
1071 : // FormatMajorVersion indicates the database's current format
1072 : // version.
1073 : FormatMajorVersion FormatMajorVersion
1074 : // ManifestFilename is the filename of the current active manifest,
1075 : // if the database exists.
1076 : ManifestFilename string
1077 : }
1078 :
1079 : // Peek looks for an existing database in dirname on the provided FS. It
1080 : // returns a brief description of the database. Peek is read-only and
1081 : // does not open the database
1082 1 : func Peek(dirname string, fs vfs.FS) (*DBDesc, error) {
1083 1 : vers, versMarker, err := lookupFormatMajorVersion(fs, dirname)
1084 1 : if err != nil {
1085 1 : return nil, err
1086 1 : }
1087 : // TODO(jackson): Immediately closing the marker is clunky. Add a
1088 : // PeekMarker variant that avoids opening the directory.
1089 1 : if err := versMarker.Close(); err != nil {
1090 0 : return nil, err
1091 0 : }
1092 :
1093 : // Find the currently active manifest, if there is one.
1094 1 : manifestMarker, manifestFileNum, exists, err := findCurrentManifest(fs, dirname)
1095 1 : if err != nil {
1096 0 : return nil, err
1097 0 : }
1098 : // TODO(jackson): Immediately closing the marker is clunky. Add a
1099 : // PeekMarker variant that avoids opening the directory.
1100 1 : if err := manifestMarker.Close(); err != nil {
1101 0 : return nil, err
1102 0 : }
1103 :
1104 1 : desc := &DBDesc{
1105 1 : Exists: exists,
1106 1 : FormatMajorVersion: vers,
1107 1 : }
1108 1 : if exists {
1109 1 : desc.ManifestFilename = base.MakeFilepath(fs, dirname, fileTypeManifest, manifestFileNum)
1110 1 : }
1111 1 : return desc, nil
1112 : }
1113 :
1114 : // LockDirectory acquires the database directory lock in the named directory,
1115 : // preventing another process from opening the database. LockDirectory returns a
1116 : // handle to the held lock that may be passed to Open through Options.Lock to
1117 : // subsequently open the database, skipping lock acquistion during Open.
1118 : //
1119 : // LockDirectory may be used to expand the critical section protected by the
1120 : // database lock to include setup before the call to Open.
1121 1 : func LockDirectory(dirname string, fs vfs.FS) (*Lock, error) {
1122 1 : fileLock, err := fs.Lock(base.MakeFilepath(fs, dirname, fileTypeLock, base.DiskFileNum(0)))
1123 1 : if err != nil {
1124 1 : return nil, err
1125 1 : }
1126 1 : l := &Lock{dirname: dirname, fileLock: fileLock}
1127 1 : l.refs.Store(1)
1128 1 : invariants.SetFinalizer(l, func(obj interface{}) {
1129 1 : if refs := obj.(*Lock).refs.Load(); refs > 0 {
1130 0 : panic(errors.AssertionFailedf("lock for %q finalized with %d refs", dirname, refs))
1131 : }
1132 : })
1133 1 : return l, nil
1134 : }
1135 :
1136 : // Lock represents a file lock on a directory. It may be passed to Open through
1137 : // Options.Lock to elide lock aquisition during Open.
1138 : type Lock struct {
1139 : dirname string
1140 : fileLock io.Closer
1141 : // refs is a count of the number of handles on the lock. refs must be 0, 1
1142 : // or 2.
1143 : //
1144 : // When acquired by the client and passed to Open, refs = 1 and the Open
1145 : // call increments it to 2. When the database is closed, it's decremented to
1146 : // 1. Finally when the original caller, calls Close on the Lock, it's
1147 : // drecemented to zero and the underlying file lock is released.
1148 : //
1149 : // When Open acquires the file lock, refs remains at 1 until the database is
1150 : // closed.
1151 : refs atomic.Int32
1152 : }
1153 :
1154 1 : func (l *Lock) refForOpen() error {
1155 1 : // During Open, when a user passed in a lock, the reference count must be
1156 1 : // exactly 1. If it's zero, the lock is no longer held and is invalid. If
1157 1 : // it's 2, the lock is already in use by another database within the
1158 1 : // process.
1159 1 : if !l.refs.CompareAndSwap(1, 2) {
1160 1 : return errors.Errorf("pebble: unexpected Lock reference count; is the lock already in use?")
1161 1 : }
1162 1 : return nil
1163 : }
1164 :
1165 : // Close releases the lock, permitting another process to lock and open the
1166 : // database. Close must not be called until after a database using the Lock has
1167 : // been closed.
1168 1 : func (l *Lock) Close() error {
1169 1 : if l.refs.Add(-1) > 0 {
1170 1 : return nil
1171 1 : }
1172 1 : defer func() { l.fileLock = nil }()
1173 1 : return l.fileLock.Close()
1174 : }
1175 :
1176 1 : func (l *Lock) pathMatches(dirname string) error {
1177 1 : if dirname == l.dirname {
1178 1 : return nil
1179 1 : }
1180 : // Check for relative paths, symlinks, etc. This isn't ideal because we're
1181 : // circumventing the vfs.FS interface here.
1182 : //
1183 : // TODO(jackson): We could add support for retrieving file inodes through Stat
1184 : // calls in the VFS interface on platforms where it's available and use that
1185 : // to differentiate.
1186 1 : dirStat, err1 := os.Stat(dirname)
1187 1 : lockDirStat, err2 := os.Stat(l.dirname)
1188 1 : if err1 == nil && err2 == nil && os.SameFile(dirStat, lockDirStat) {
1189 1 : return nil
1190 1 : }
1191 0 : return errors.Join(
1192 0 : errors.Newf("pebble: opts.Lock acquired in %q not %q", l.dirname, dirname),
1193 0 : err1, err2)
1194 : }
1195 :
1196 : // ErrDBDoesNotExist is generated when ErrorIfNotExists is set and the database
1197 : // does not exist.
1198 : //
1199 : // Note that errors can be wrapped with more details; use errors.Is().
1200 : var ErrDBDoesNotExist = errors.New("pebble: database does not exist")
1201 :
1202 : // ErrDBAlreadyExists is generated when ErrorIfExists is set and the database
1203 : // already exists.
1204 : //
1205 : // Note that errors can be wrapped with more details; use errors.Is().
1206 : var ErrDBAlreadyExists = errors.New("pebble: database already exists")
1207 :
1208 : // ErrDBNotPristine is generated when ErrorIfNotPristine is set and the database
1209 : // already exists and is not pristine.
1210 : //
1211 : // Note that errors can be wrapped with more details; use errors.Is().
1212 : var ErrDBNotPristine = errors.New("pebble: database already exists and is not pristine")
1213 :
1214 : // IsCorruptionError returns true if the given error indicates database
1215 : // corruption.
1216 1 : func IsCorruptionError(err error) bool {
1217 1 : return errors.Is(err, base.ErrCorruption)
1218 1 : }
1219 :
1220 1 : func checkConsistency(v *manifest.Version, dirname string, objProvider objstorage.Provider) error {
1221 1 : var errs []error
1222 1 : dedup := make(map[base.DiskFileNum]struct{})
1223 1 : for level, files := range v.Levels {
1224 1 : iter := files.Iter()
1225 1 : for f := iter.First(); f != nil; f = iter.Next() {
1226 1 : backingState := f.FileBacking
1227 1 : if _, ok := dedup[backingState.DiskFileNum]; ok {
1228 1 : continue
1229 : }
1230 1 : dedup[backingState.DiskFileNum] = struct{}{}
1231 1 : fileNum := backingState.DiskFileNum
1232 1 : fileSize := backingState.Size
1233 1 : // We skip over remote objects; those are instead checked asynchronously
1234 1 : // by the table stats loading job.
1235 1 : meta, err := objProvider.Lookup(base.FileTypeTable, fileNum)
1236 1 : var size int64
1237 1 : if err == nil {
1238 1 : if meta.IsRemote() {
1239 1 : continue
1240 : }
1241 1 : size, err = objProvider.Size(meta)
1242 : }
1243 1 : if err != nil {
1244 1 : errs = append(errs, errors.Wrapf(err, "L%d: %s", errors.Safe(level), fileNum))
1245 1 : continue
1246 : }
1247 :
1248 1 : if size != int64(fileSize) {
1249 0 : errs = append(errs, errors.Errorf(
1250 0 : "L%d: %s: object size mismatch (%s): %d (disk) != %d (MANIFEST)",
1251 0 : errors.Safe(level), fileNum, objProvider.Path(meta),
1252 0 : errors.Safe(size), errors.Safe(fileSize)))
1253 0 : continue
1254 : }
1255 : }
1256 : }
1257 1 : return errors.Join(errs...)
1258 : }
1259 :
1260 : type walEventListenerAdaptor struct {
1261 : l *EventListener
1262 : }
1263 :
1264 1 : func (l walEventListenerAdaptor) LogCreated(ci wal.CreateInfo) {
1265 1 : // TODO(sumeer): extend WALCreateInfo for the failover case in case the path
1266 1 : // is insufficient to infer whether primary or secondary.
1267 1 : wci := WALCreateInfo{
1268 1 : JobID: ci.JobID,
1269 1 : Path: ci.Path,
1270 1 : FileNum: base.DiskFileNum(ci.Num),
1271 1 : RecycledFileNum: ci.RecycledFileNum,
1272 1 : Err: ci.Err,
1273 1 : }
1274 1 : l.l.WALCreated(wci)
1275 1 : }
|