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