Refactor replica system
This commit is contained in:
633
db.go
633
db.go
@@ -16,14 +16,13 @@ import (
|
||||
"math/rand"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/benbjohnson/litestream/internal"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/client_golang/prometheus/promauto"
|
||||
"golang.org/x/sync/errgroup"
|
||||
)
|
||||
|
||||
// Default DB settings.
|
||||
@@ -51,10 +50,8 @@ type DB struct {
|
||||
pageSize int // page size, in bytes
|
||||
notify chan struct{} // closes on WAL change
|
||||
|
||||
uid, gid int // db user/group obtained on init
|
||||
mode os.FileMode
|
||||
diruid, dirgid int // db parent user/group obtained on init
|
||||
dirmode os.FileMode
|
||||
fileInfo os.FileInfo // db info cached during init
|
||||
dirInfo os.FileInfo // parent dir info cached during init
|
||||
|
||||
ctx context.Context
|
||||
cancel func()
|
||||
@@ -96,7 +93,7 @@ type DB struct {
|
||||
|
||||
// List of replicas for the database.
|
||||
// Must be set before calling Open().
|
||||
Replicas []Replica
|
||||
Replicas []*Replica
|
||||
}
|
||||
|
||||
// NewDB returns a new instance of DB for a given path.
|
||||
@@ -104,8 +101,6 @@ func NewDB(path string) *DB {
|
||||
db := &DB{
|
||||
path: path,
|
||||
notify: make(chan struct{}),
|
||||
uid: -1, gid: -1, mode: 0600,
|
||||
diruid: -1, dirgid: -1, dirmode: 0700,
|
||||
|
||||
MinCheckpointPageN: DefaultMinCheckpointPageN,
|
||||
MaxCheckpointPageN: DefaultMaxCheckpointPageN,
|
||||
@@ -196,10 +191,7 @@ func (db *DB) CurrentShadowWALIndex(generation string) (index int, size int64, e
|
||||
|
||||
// Find highest wal index.
|
||||
for _, fi := range fis {
|
||||
if !strings.HasSuffix(fi.Name(), WALExt) {
|
||||
continue
|
||||
}
|
||||
if v, _, _, err := ParseWALPath(fi.Name()); err != nil {
|
||||
if v, err := ParseWALPath(fi.Name()); err != nil {
|
||||
continue // invalid wal filename
|
||||
} else if v > index {
|
||||
index = v
|
||||
@@ -210,8 +202,18 @@ func (db *DB) CurrentShadowWALIndex(generation string) (index int, size int64, e
|
||||
return index, size, nil
|
||||
}
|
||||
|
||||
// FileInfo returns the cached file stats for the database file when it was initialized.
|
||||
func (db *DB) FileInfo() os.FileInfo {
|
||||
return db.fileInfo
|
||||
}
|
||||
|
||||
// DirInfo returns the cached file stats for the parent directory of the database file when it was initialized.
|
||||
func (db *DB) DirInfo() os.FileInfo {
|
||||
return db.dirInfo
|
||||
}
|
||||
|
||||
// Replica returns a replica by name.
|
||||
func (db *DB) Replica(name string) Replica {
|
||||
func (db *DB) Replica(name string) *Replica {
|
||||
for _, r := range db.Replicas {
|
||||
if r.Name() == name {
|
||||
return r
|
||||
@@ -364,42 +366,6 @@ func (db *DB) UpdatedAt() (time.Time, error) {
|
||||
return t, nil
|
||||
}
|
||||
|
||||
// Snapshots returns a list of all snapshots across all replicas.
|
||||
func (db *DB) Snapshots(ctx context.Context) ([]*SnapshotInfo, error) {
|
||||
var infos []*SnapshotInfo
|
||||
for _, r := range db.Replicas {
|
||||
a, err := r.Snapshots(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
infos = append(infos, a...)
|
||||
}
|
||||
|
||||
// Sort in order by time.
|
||||
sort.Slice(infos, func(i, j int) bool {
|
||||
return infos[i].CreatedAt.Before(infos[j].CreatedAt)
|
||||
})
|
||||
return infos, nil
|
||||
}
|
||||
|
||||
// WALs returns a list of all WAL files across all replicas.
|
||||
func (db *DB) WALs(ctx context.Context) ([]*WALInfo, error) {
|
||||
var infos []*WALInfo
|
||||
for _, r := range db.Replicas {
|
||||
a, err := r.WALs(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
infos = append(infos, a...)
|
||||
}
|
||||
|
||||
// Sort in order by time.
|
||||
sort.Slice(infos, func(i, j int) bool {
|
||||
return infos[i].CreatedAt.Before(infos[j].CreatedAt)
|
||||
})
|
||||
return infos, nil
|
||||
}
|
||||
|
||||
// init initializes the connection to the database.
|
||||
// Skipped if already initialized or if the database file does not exist.
|
||||
func (db *DB) init() (err error) {
|
||||
@@ -415,15 +381,13 @@ func (db *DB) init() (err error) {
|
||||
} else if err != nil {
|
||||
return err
|
||||
}
|
||||
db.uid, db.gid = fileinfo(fi)
|
||||
db.mode = fi.Mode()
|
||||
db.fileInfo = fi
|
||||
|
||||
// Obtain permissions for parent directory.
|
||||
if fi, err = os.Stat(filepath.Dir(db.path)); err != nil {
|
||||
return err
|
||||
}
|
||||
db.diruid, db.dirgid = fileinfo(fi)
|
||||
db.dirmode = fi.Mode()
|
||||
db.dirInfo = fi
|
||||
|
||||
dsn := db.path
|
||||
dsn += fmt.Sprintf("?_busy_timeout=%d", BusyTimeout.Milliseconds())
|
||||
@@ -489,7 +453,7 @@ func (db *DB) init() (err error) {
|
||||
}
|
||||
|
||||
// Ensure meta directory structure exists.
|
||||
if err := mkdirAll(db.MetaPath(), db.dirmode, db.diruid, db.dirgid); err != nil {
|
||||
if err := internal.MkdirAll(db.MetaPath(), db.dirInfo); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
@@ -596,7 +560,7 @@ func (db *DB) cleanWAL() error {
|
||||
// Determine lowest index that's been replicated to all replicas.
|
||||
min := -1
|
||||
for _, r := range db.Replicas {
|
||||
pos := r.LastPos()
|
||||
pos := r.Pos()
|
||||
if pos.Generation != generation {
|
||||
pos = Pos{} // different generation, reset index to zero
|
||||
}
|
||||
@@ -620,7 +584,7 @@ func (db *DB) cleanWAL() error {
|
||||
return err
|
||||
}
|
||||
for _, fi := range fis {
|
||||
if idx, _, _, err := ParseWALPath(fi.Name()); err != nil || idx >= min {
|
||||
if idx, err := ParseWALPath(fi.Name()); err != nil || idx >= min {
|
||||
continue
|
||||
}
|
||||
if err := os.Remove(filepath.Join(dir, fi.Name())); err != nil {
|
||||
@@ -696,7 +660,7 @@ func (db *DB) createGeneration() (string, error) {
|
||||
|
||||
// Generate new directory.
|
||||
dir := filepath.Join(db.MetaPath(), "generations", generation)
|
||||
if err := mkdirAll(dir, db.dirmode, db.diruid, db.dirgid); err != nil {
|
||||
if err := internal.MkdirAll(dir, db.dirInfo); err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
@@ -707,10 +671,15 @@ func (db *DB) createGeneration() (string, error) {
|
||||
|
||||
// Atomically write generation name as current generation.
|
||||
generationNamePath := db.GenerationNamePath()
|
||||
if err := ioutil.WriteFile(generationNamePath+".tmp", []byte(generation+"\n"), db.mode); err != nil {
|
||||
mode := os.FileMode(0600)
|
||||
if db.fileInfo != nil {
|
||||
mode = db.fileInfo.Mode()
|
||||
}
|
||||
if err := ioutil.WriteFile(generationNamePath+".tmp", []byte(generation+"\n"), mode); err != nil {
|
||||
return "", fmt.Errorf("write generation temp file: %w", err)
|
||||
}
|
||||
_ = os.Chown(generationNamePath+".tmp", db.uid, db.gid)
|
||||
uid, gid := internal.Fileinfo(db.fileInfo)
|
||||
_ = os.Chown(generationNamePath+".tmp", uid, gid)
|
||||
if err := os.Rename(generationNamePath+".tmp", generationNamePath); err != nil {
|
||||
return "", fmt.Errorf("rename generation file: %w", err)
|
||||
}
|
||||
@@ -801,7 +770,7 @@ func (db *DB) Sync(ctx context.Context) (err error) {
|
||||
if checkpoint {
|
||||
changed = true
|
||||
|
||||
if err := db.checkpointAndInit(ctx, info.generation, checkpointMode); err != nil {
|
||||
if err := db.checkpoint(ctx, info.generation, checkpointMode); err != nil {
|
||||
return fmt.Errorf("checkpoint: mode=%v err=%w", checkpointMode, err)
|
||||
}
|
||||
}
|
||||
@@ -959,7 +928,7 @@ func (db *DB) syncWAL(info syncInfo) (newSize int64, err error) {
|
||||
|
||||
// Parse index of current shadow WAL file.
|
||||
dir, base := filepath.Split(info.shadowWALPath)
|
||||
index, _, _, err := ParseWALPath(base)
|
||||
index, err := ParseWALPath(base)
|
||||
if err != nil {
|
||||
return 0, fmt.Errorf("cannot parse shadow wal filename: %s", base)
|
||||
}
|
||||
@@ -993,12 +962,17 @@ func (db *DB) initShadowWALFile(filename string) (int64, error) {
|
||||
}
|
||||
|
||||
// Write header to new WAL shadow file.
|
||||
if err := mkdirAll(filepath.Dir(filename), db.dirmode, db.diruid, db.dirgid); err != nil {
|
||||
mode := os.FileMode(0600)
|
||||
if fi := db.fileInfo; fi != nil {
|
||||
mode = fi.Mode()
|
||||
}
|
||||
if err := internal.MkdirAll(filepath.Dir(filename), db.dirInfo); err != nil {
|
||||
return 0, err
|
||||
} else if err := ioutil.WriteFile(filename, hdr, db.mode); err != nil {
|
||||
} else if err := ioutil.WriteFile(filename, hdr, mode); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
_ = os.Chown(filename, db.uid, db.gid)
|
||||
uid, gid := internal.Fileinfo(db.fileInfo)
|
||||
_ = os.Chown(filename, uid, gid)
|
||||
|
||||
// Copy as much shadow WAL as available.
|
||||
newSize, err := db.copyToShadowWAL(filename)
|
||||
@@ -1155,7 +1129,7 @@ func (db *DB) shadowWALReader(pos Pos) (r *ShadowWALReader, err error) {
|
||||
// Ensure file is closed if any error occurs.
|
||||
defer func() {
|
||||
if err != nil {
|
||||
r.Close()
|
||||
f.Close()
|
||||
}
|
||||
}()
|
||||
|
||||
@@ -1258,13 +1232,91 @@ func readLastChecksumFrom(f *os.File, pageSize int) (uint32, uint32, error) {
|
||||
}
|
||||
|
||||
// Checkpoint performs a checkpoint on the WAL file.
|
||||
func (db *DB) Checkpoint(mode string) (err error) {
|
||||
func (db *DB) Checkpoint(ctx context.Context, mode string) (err error) {
|
||||
db.mu.Lock()
|
||||
defer db.mu.Unlock()
|
||||
return db.checkpoint(mode)
|
||||
|
||||
generation, err := db.CurrentGeneration()
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot determine generation: %w", err)
|
||||
}
|
||||
return db.checkpoint(ctx, generation, mode)
|
||||
}
|
||||
|
||||
func (db *DB) checkpoint(mode string) (err error) {
|
||||
// checkpointAndInit performs a checkpoint on the WAL file and initializes a
|
||||
// new shadow WAL file.
|
||||
func (db *DB) checkpoint(ctx context.Context, generation, mode string) error {
|
||||
shadowWALPath, err := db.CurrentShadowWALPath(generation)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Read WAL header before checkpoint to check if it has been restarted.
|
||||
hdr, err := readWALHeader(db.WALPath())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Copy shadow WAL before checkpoint to copy as much as possible.
|
||||
if _, err := db.copyToShadowWAL(shadowWALPath); err != nil {
|
||||
return fmt.Errorf("cannot copy to end of shadow wal before checkpoint: %w", err)
|
||||
}
|
||||
|
||||
// Execute checkpoint and immediately issue a write to the WAL to ensure
|
||||
// a new page is written.
|
||||
if err := db.execCheckpoint(mode); err != nil {
|
||||
return err
|
||||
} else if _, err = db.db.Exec(`INSERT INTO _litestream_seq (id, seq) VALUES (1, 1) ON CONFLICT (id) DO UPDATE SET seq = seq + 1`); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// If WAL hasn't been restarted, exit.
|
||||
if other, err := readWALHeader(db.WALPath()); err != nil {
|
||||
return err
|
||||
} else if bytes.Equal(hdr, other) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Start a transaction. This will be promoted immediately after.
|
||||
tx, err := db.db.Begin()
|
||||
if err != nil {
|
||||
return fmt.Errorf("begin: %w", err)
|
||||
}
|
||||
defer func() { _ = rollback(tx) }()
|
||||
|
||||
// Insert into the lock table to promote to a write tx. The lock table
|
||||
// insert will never actually occur because our tx will be rolled back,
|
||||
// however, it will ensure our tx grabs the write lock. Unfortunately,
|
||||
// we can't call "BEGIN IMMEDIATE" as we are already in a transaction.
|
||||
if _, err := tx.ExecContext(ctx, `INSERT INTO _litestream_lock (id) VALUES (1);`); err != nil {
|
||||
return fmt.Errorf("_litestream_lock: %w", err)
|
||||
}
|
||||
|
||||
// Copy the end of the previous WAL before starting a new shadow WAL.
|
||||
if _, err := db.copyToShadowWAL(shadowWALPath); err != nil {
|
||||
return fmt.Errorf("cannot copy to end of shadow wal: %w", err)
|
||||
}
|
||||
|
||||
// Parse index of current shadow WAL file.
|
||||
index, err := ParseWALPath(shadowWALPath)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot parse shadow wal filename: %s", shadowWALPath)
|
||||
}
|
||||
|
||||
// Start a new shadow WAL file with next index.
|
||||
newShadowWALPath := filepath.Join(filepath.Dir(shadowWALPath), FormatWALPath(index+1))
|
||||
if _, err := db.initShadowWALFile(newShadowWALPath); err != nil {
|
||||
return fmt.Errorf("cannot init shadow wal file: name=%s err=%w", newShadowWALPath, err)
|
||||
}
|
||||
|
||||
// Release write lock before checkpointing & exiting.
|
||||
if err := tx.Rollback(); err != nil {
|
||||
return fmt.Errorf("rollback post-checkpoint tx: %w", err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (db *DB) execCheckpoint(mode string) (err error) {
|
||||
// Ignore if there is no underlying database.
|
||||
if db.db == nil {
|
||||
return nil
|
||||
@@ -1310,79 +1362,6 @@ func (db *DB) checkpoint(mode string) (err error) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// checkpointAndInit performs a checkpoint on the WAL file and initializes a
|
||||
// new shadow WAL file.
|
||||
func (db *DB) checkpointAndInit(ctx context.Context, generation, mode string) error {
|
||||
shadowWALPath, err := db.CurrentShadowWALPath(generation)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Read WAL header before checkpoint to check if it has been restarted.
|
||||
hdr, err := readWALHeader(db.WALPath())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Copy shadow WAL before checkpoint to copy as much as possible.
|
||||
if _, err := db.copyToShadowWAL(shadowWALPath); err != nil {
|
||||
return fmt.Errorf("cannot copy to end of shadow wal before checkpoint: %w", err)
|
||||
}
|
||||
|
||||
// Execute checkpoint and immediately issue a write to the WAL to ensure
|
||||
// a new page is written.
|
||||
if err := db.checkpoint(mode); err != nil {
|
||||
return err
|
||||
} else if _, err = db.db.Exec(`INSERT INTO _litestream_seq (id, seq) VALUES (1, 1) ON CONFLICT (id) DO UPDATE SET seq = seq + 1`); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// If WAL hasn't been restarted, exit.
|
||||
if other, err := readWALHeader(db.WALPath()); err != nil {
|
||||
return err
|
||||
} else if bytes.Equal(hdr, other) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Start a transaction. This will be promoted immediately after.
|
||||
tx, err := db.db.Begin()
|
||||
if err != nil {
|
||||
return fmt.Errorf("begin: %w", err)
|
||||
}
|
||||
defer func() { _ = rollback(tx) }()
|
||||
|
||||
// Insert into the lock table to promote to a write tx. The lock table
|
||||
// insert will never actually occur because our tx will be rolled back,
|
||||
// however, it will ensure our tx grabs the write lock. Unfortunately,
|
||||
// we can't call "BEGIN IMMEDIATE" as we are already in a transaction.
|
||||
if _, err := tx.ExecContext(ctx, `INSERT INTO _litestream_lock (id) VALUES (1);`); err != nil {
|
||||
return fmt.Errorf("_litestream_lock: %w", err)
|
||||
}
|
||||
|
||||
// Copy the end of the previous WAL before starting a new shadow WAL.
|
||||
if _, err := db.copyToShadowWAL(shadowWALPath); err != nil {
|
||||
return fmt.Errorf("cannot copy to end of shadow wal: %w", err)
|
||||
}
|
||||
|
||||
// Parse index of current shadow WAL file.
|
||||
index, _, _, err := ParseWALPath(shadowWALPath)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot parse shadow wal filename: %s", shadowWALPath)
|
||||
}
|
||||
|
||||
// Start a new shadow WAL file with next index.
|
||||
newShadowWALPath := filepath.Join(filepath.Dir(shadowWALPath), FormatWALPath(index+1))
|
||||
if _, err := db.initShadowWALFile(newShadowWALPath); err != nil {
|
||||
return fmt.Errorf("cannot init shadow wal file: name=%s err=%w", newShadowWALPath, err)
|
||||
}
|
||||
|
||||
// Release write lock before checkpointing & exiting.
|
||||
if err := tx.Rollback(); err != nil {
|
||||
return fmt.Errorf("rollback post-checkpoint tx: %w", err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// monitor runs in a separate goroutine and monitors the database & WAL.
|
||||
func (db *DB) monitor() {
|
||||
ticker := time.NewTicker(db.MonitorInterval)
|
||||
@@ -1403,189 +1382,12 @@ func (db *DB) monitor() {
|
||||
}
|
||||
}
|
||||
|
||||
// RestoreReplica restores the database from a replica based on the options given.
|
||||
// This method will restore into opt.OutputPath, if specified, or into the
|
||||
// DB's original database path. It can optionally restore from a specific
|
||||
// replica or generation or it will automatically choose the best one. Finally,
|
||||
// a timestamp can be specified to restore the database to a specific
|
||||
// point-in-time.
|
||||
func RestoreReplica(ctx context.Context, r Replica, opt RestoreOptions) (err error) {
|
||||
// Validate options.
|
||||
if opt.OutputPath == "" {
|
||||
return fmt.Errorf("output path required")
|
||||
} else if opt.Generation == "" && opt.Index != math.MaxInt32 {
|
||||
return fmt.Errorf("must specify generation when restoring to index")
|
||||
} else if opt.Index != math.MaxInt32 && !opt.Timestamp.IsZero() {
|
||||
return fmt.Errorf("cannot specify index & timestamp to restore")
|
||||
}
|
||||
|
||||
// Ensure logger exists.
|
||||
logger := opt.Logger
|
||||
if logger == nil {
|
||||
logger = log.New(ioutil.Discard, "", 0)
|
||||
}
|
||||
|
||||
logPrefix := r.Name()
|
||||
if db := r.DB(); db != nil {
|
||||
logPrefix = fmt.Sprintf("%s(%s)", db.Path(), r.Name())
|
||||
}
|
||||
|
||||
// Ensure output path does not already exist.
|
||||
if _, err := os.Stat(opt.OutputPath); err == nil {
|
||||
return fmt.Errorf("cannot restore, output path already exists: %s", opt.OutputPath)
|
||||
} else if err != nil && !os.IsNotExist(err) {
|
||||
return err
|
||||
}
|
||||
|
||||
// Find lastest snapshot that occurs before timestamp or index.
|
||||
var minWALIndex int
|
||||
if opt.Index < math.MaxInt32 {
|
||||
if minWALIndex, err = SnapshotIndexByIndex(ctx, r, opt.Generation, opt.Index); err != nil {
|
||||
return fmt.Errorf("cannot find snapshot index: %w", err)
|
||||
}
|
||||
} else {
|
||||
if minWALIndex, err = SnapshotIndexAt(ctx, r, opt.Generation, opt.Timestamp); err != nil {
|
||||
return fmt.Errorf("cannot find snapshot index by timestamp: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
// Find the maximum WAL index that occurs before timestamp.
|
||||
maxWALIndex, err := WALIndexAt(ctx, r, opt.Generation, opt.Index, opt.Timestamp)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot find max wal index for restore: %w", err)
|
||||
}
|
||||
snapshotOnly := maxWALIndex == -1
|
||||
|
||||
// Initialize starting position.
|
||||
pos := Pos{Generation: opt.Generation, Index: minWALIndex}
|
||||
tmpPath := opt.OutputPath + ".tmp"
|
||||
|
||||
// Copy snapshot to output path.
|
||||
logger.Printf("%s: restoring snapshot %s/%08x to %s", logPrefix, opt.Generation, minWALIndex, tmpPath)
|
||||
if err := restoreSnapshot(ctx, r, pos.Generation, pos.Index, tmpPath); err != nil {
|
||||
return fmt.Errorf("cannot restore snapshot: %w", err)
|
||||
}
|
||||
|
||||
// If no WAL files available, move snapshot to final path & exit early.
|
||||
if snapshotOnly {
|
||||
logger.Printf("%s: snapshot only, finalizing database", logPrefix)
|
||||
return os.Rename(tmpPath, opt.OutputPath)
|
||||
}
|
||||
|
||||
// Begin processing WAL files.
|
||||
logger.Printf("%s: restoring wal files: generation=%s index=[%08x,%08x]", logPrefix, opt.Generation, minWALIndex, maxWALIndex)
|
||||
|
||||
// Fill input channel with all WAL indexes to be loaded in order.
|
||||
ch := make(chan int, maxWALIndex-minWALIndex+1)
|
||||
for index := minWALIndex; index <= maxWALIndex; index++ {
|
||||
ch <- index
|
||||
}
|
||||
close(ch)
|
||||
|
||||
// Track load state for each WAL.
|
||||
var mu sync.Mutex
|
||||
cond := sync.NewCond(&mu)
|
||||
walStates := make([]walRestoreState, maxWALIndex-minWALIndex+1)
|
||||
|
||||
parallelism := opt.Parallelism
|
||||
if parallelism < 1 {
|
||||
parallelism = 1
|
||||
}
|
||||
|
||||
// Download WAL files to disk in parallel.
|
||||
g, ctx := errgroup.WithContext(ctx)
|
||||
for i := 0; i < parallelism; i++ {
|
||||
g.Go(func() error {
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
cond.Broadcast()
|
||||
return err
|
||||
case index, ok := <-ch:
|
||||
if !ok {
|
||||
cond.Broadcast()
|
||||
return nil
|
||||
}
|
||||
|
||||
startTime := time.Now()
|
||||
|
||||
err := downloadWAL(ctx, r, opt.Generation, index, tmpPath)
|
||||
if err != nil {
|
||||
err = fmt.Errorf("cannot download wal %s/%08x: %w", opt.Generation, index, err)
|
||||
}
|
||||
|
||||
// Mark index as ready-to-apply and notify applying code.
|
||||
mu.Lock()
|
||||
walStates[index-minWALIndex] = walRestoreState{ready: true, err: err}
|
||||
mu.Unlock()
|
||||
cond.Broadcast()
|
||||
|
||||
// Returning the error here will cancel the other goroutines.
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
logger.Printf("%s: downloaded wal %s/%08x elapsed=%s",
|
||||
logPrefix, opt.Generation, index,
|
||||
time.Since(startTime).String(),
|
||||
)
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// Apply WAL files in order as they are ready.
|
||||
for index := minWALIndex; index <= maxWALIndex; index++ {
|
||||
// Wait until next WAL file is ready to apply.
|
||||
mu.Lock()
|
||||
for !walStates[index-minWALIndex].ready {
|
||||
if err := ctx.Err(); err != nil {
|
||||
return err
|
||||
}
|
||||
cond.Wait()
|
||||
}
|
||||
if err := walStates[index-minWALIndex].err; err != nil {
|
||||
return err
|
||||
}
|
||||
mu.Unlock()
|
||||
|
||||
// Apply WAL to database file.
|
||||
startTime := time.Now()
|
||||
if err = applyWAL(ctx, index, tmpPath); err != nil {
|
||||
return fmt.Errorf("cannot apply wal: %w", err)
|
||||
}
|
||||
logger.Printf("%s: applied wal %s/%08x elapsed=%s",
|
||||
logPrefix, opt.Generation, index,
|
||||
time.Since(startTime).String(),
|
||||
)
|
||||
}
|
||||
|
||||
// Ensure all goroutines finish. All errors should have been handled during
|
||||
// the processing of WAL files but this ensures that all processing is done.
|
||||
if err := g.Wait(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Copy file to final location.
|
||||
logger.Printf("%s: renaming database from temporary location", logPrefix)
|
||||
if err := os.Rename(tmpPath, opt.OutputPath); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
type walRestoreState struct {
|
||||
ready bool
|
||||
err error
|
||||
}
|
||||
|
||||
// CalcRestoreTarget returns a replica & generation to restore from based on opt criteria.
|
||||
func (db *DB) CalcRestoreTarget(ctx context.Context, opt RestoreOptions) (Replica, string, error) {
|
||||
func (db *DB) CalcRestoreTarget(ctx context.Context, opt RestoreOptions) (*Replica, string, error) {
|
||||
var target struct {
|
||||
replica Replica
|
||||
replica *Replica
|
||||
generation string
|
||||
stats GenerationStats
|
||||
updatedAt time.Time
|
||||
}
|
||||
|
||||
for _, r := range db.Replicas {
|
||||
@@ -1594,134 +1396,21 @@ func (db *DB) CalcRestoreTarget(ctx context.Context, opt RestoreOptions) (Replic
|
||||
continue
|
||||
}
|
||||
|
||||
generation, stats, err := CalcReplicaRestoreTarget(ctx, r, opt)
|
||||
generation, updatedAt, err := r.CalcRestoreTarget(ctx, opt)
|
||||
if err != nil {
|
||||
return nil, "", err
|
||||
}
|
||||
|
||||
// Use the latest replica if we have multiple candidates.
|
||||
if !stats.UpdatedAt.After(target.stats.UpdatedAt) {
|
||||
if !updatedAt.After(target.updatedAt) {
|
||||
continue
|
||||
}
|
||||
|
||||
target.replica, target.generation, target.stats = r, generation, stats
|
||||
target.replica, target.generation, target.updatedAt = r, generation, updatedAt
|
||||
}
|
||||
return target.replica, target.generation, nil
|
||||
}
|
||||
|
||||
// CalcReplicaRestoreTarget returns a generation to restore from.
|
||||
func CalcReplicaRestoreTarget(ctx context.Context, r Replica, opt RestoreOptions) (generation string, stats GenerationStats, err error) {
|
||||
var target struct {
|
||||
generation string
|
||||
stats GenerationStats
|
||||
}
|
||||
|
||||
generations, err := r.Generations(ctx)
|
||||
if err != nil {
|
||||
return "", stats, fmt.Errorf("cannot fetch generations: %w", err)
|
||||
}
|
||||
|
||||
// Search generations for one that contains the requested timestamp.
|
||||
for _, generation := range generations {
|
||||
// Skip generation if it does not match filter.
|
||||
if opt.Generation != "" && generation != opt.Generation {
|
||||
continue
|
||||
}
|
||||
|
||||
// Fetch stats for generation.
|
||||
stats, err := r.GenerationStats(ctx, generation)
|
||||
if err != nil {
|
||||
return "", stats, fmt.Errorf("cannot determine stats for generation (%s/%s): %s", r.Name(), generation, err)
|
||||
}
|
||||
|
||||
// Skip if it does not contain timestamp.
|
||||
if !opt.Timestamp.IsZero() {
|
||||
if opt.Timestamp.Before(stats.CreatedAt) || opt.Timestamp.After(stats.UpdatedAt) {
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
// Use the latest replica if we have multiple candidates.
|
||||
if !stats.UpdatedAt.After(target.stats.UpdatedAt) {
|
||||
continue
|
||||
}
|
||||
|
||||
target.generation = generation
|
||||
target.stats = stats
|
||||
}
|
||||
|
||||
return target.generation, target.stats, nil
|
||||
}
|
||||
|
||||
// restoreSnapshot copies a snapshot from the replica to a file.
|
||||
func restoreSnapshot(ctx context.Context, r Replica, generation string, index int, filename string) error {
|
||||
// Determine the user/group & mode based on the DB, if available.
|
||||
uid, gid, mode := -1, -1, os.FileMode(0600)
|
||||
diruid, dirgid, dirmode := -1, -1, os.FileMode(0700)
|
||||
if db := r.DB(); db != nil {
|
||||
uid, gid, mode = db.uid, db.gid, db.mode
|
||||
diruid, dirgid, dirmode = db.diruid, db.dirgid, db.dirmode
|
||||
}
|
||||
|
||||
if err := mkdirAll(filepath.Dir(filename), dirmode, diruid, dirgid); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
f, err := createFile(filename, mode, uid, gid)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer f.Close()
|
||||
|
||||
rd, err := r.SnapshotReader(ctx, generation, index)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer rd.Close()
|
||||
|
||||
if _, err := io.Copy(f, rd); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := f.Sync(); err != nil {
|
||||
return err
|
||||
}
|
||||
return f.Close()
|
||||
}
|
||||
|
||||
// downloadWAL copies a WAL file from the replica to a local copy next to the DB.
|
||||
// The WAL is later applied by applyWAL(). This function can be run in parallel
|
||||
// to download multiple WAL files simultaneously.
|
||||
func downloadWAL(ctx context.Context, r Replica, generation string, index int, dbPath string) error {
|
||||
// Determine the user/group & mode based on the DB, if available.
|
||||
uid, gid, mode := -1, -1, os.FileMode(0600)
|
||||
if db := r.DB(); db != nil {
|
||||
uid, gid, mode = db.uid, db.gid, db.mode
|
||||
}
|
||||
|
||||
// Open WAL file from replica.
|
||||
rd, err := r.WALReader(ctx, generation, index)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer rd.Close()
|
||||
|
||||
// Open handle to destination WAL path.
|
||||
f, err := createFile(fmt.Sprintf("%s-%08x-wal", dbPath, index), mode, uid, gid)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer f.Close()
|
||||
|
||||
// Copy WAL to target path.
|
||||
if _, err := io.Copy(f, rd); err != nil {
|
||||
return err
|
||||
} else if err := f.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// applyWAL performs a truncating checkpoint on the given database.
|
||||
func applyWAL(ctx context.Context, index int, dbPath string) error {
|
||||
// Copy WAL file from it's staging path to the correct "-wal" location.
|
||||
@@ -1770,7 +1459,7 @@ func (db *DB) CRC64(ctx context.Context) (uint64, Pos, error) {
|
||||
}
|
||||
|
||||
// Force a RESTART checkpoint to ensure the database is at the start of the WAL.
|
||||
if err := db.checkpointAndInit(ctx, generation, CheckpointModeRestart); err != nil {
|
||||
if err := db.checkpoint(ctx, generation, CheckpointModeRestart); err != nil {
|
||||
return 0, Pos{}, err
|
||||
}
|
||||
|
||||
@@ -1836,80 +1525,58 @@ func NewRestoreOptions() RestoreOptions {
|
||||
// Database metrics.
|
||||
var (
|
||||
dbSizeGaugeVec = promauto.NewGaugeVec(prometheus.GaugeOpts{
|
||||
Namespace: "litestream",
|
||||
Subsystem: "db",
|
||||
Name: "db_size",
|
||||
Help: "The current size of the real DB",
|
||||
Name: "litestream_db_size",
|
||||
Help: "The current size of the real DB",
|
||||
}, []string{"db"})
|
||||
|
||||
walSizeGaugeVec = promauto.NewGaugeVec(prometheus.GaugeOpts{
|
||||
Namespace: "litestream",
|
||||
Subsystem: "db",
|
||||
Name: "wal_size",
|
||||
Help: "The current size of the real WAL",
|
||||
Name: "litestream_wal_size",
|
||||
Help: "The current size of the real WAL",
|
||||
}, []string{"db"})
|
||||
|
||||
totalWALBytesCounterVec = promauto.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: "litestream",
|
||||
Subsystem: "db",
|
||||
Name: "total_wal_bytes",
|
||||
Help: "Total number of bytes written to shadow WAL",
|
||||
Name: "litestream_total_wal_bytes",
|
||||
Help: "Total number of bytes written to shadow WAL",
|
||||
}, []string{"db"})
|
||||
|
||||
shadowWALIndexGaugeVec = promauto.NewGaugeVec(prometheus.GaugeOpts{
|
||||
Namespace: "litestream",
|
||||
Subsystem: "db",
|
||||
Name: "shadow_wal_index",
|
||||
Help: "The current index of the shadow WAL",
|
||||
Name: "litestream_shadow_wal_index",
|
||||
Help: "The current index of the shadow WAL",
|
||||
}, []string{"db"})
|
||||
|
||||
shadowWALSizeGaugeVec = promauto.NewGaugeVec(prometheus.GaugeOpts{
|
||||
Namespace: "litestream",
|
||||
Subsystem: "db",
|
||||
Name: "shadow_wal_size",
|
||||
Help: "Current size of shadow WAL, in bytes",
|
||||
Name: "litestream_shadow_wal_size",
|
||||
Help: "Current size of shadow WAL, in bytes",
|
||||
}, []string{"db"})
|
||||
|
||||
syncNCounterVec = promauto.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: "litestream",
|
||||
Subsystem: "db",
|
||||
Name: "sync_count",
|
||||
Help: "Number of sync operations performed",
|
||||
Name: "litestream_sync_count",
|
||||
Help: "Number of sync operations performed",
|
||||
}, []string{"db"})
|
||||
|
||||
syncErrorNCounterVec = promauto.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: "litestream",
|
||||
Subsystem: "db",
|
||||
Name: "sync_error_count",
|
||||
Help: "Number of sync errors that have occurred",
|
||||
Name: "litestream_sync_error_count",
|
||||
Help: "Number of sync errors that have occurred",
|
||||
}, []string{"db"})
|
||||
|
||||
syncSecondsCounterVec = promauto.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: "litestream",
|
||||
Subsystem: "db",
|
||||
Name: "sync_seconds",
|
||||
Help: "Time spent syncing shadow WAL, in seconds",
|
||||
Name: "litestream_sync_seconds",
|
||||
Help: "Time spent syncing shadow WAL, in seconds",
|
||||
}, []string{"db"})
|
||||
|
||||
checkpointNCounterVec = promauto.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: "litestream",
|
||||
Subsystem: "db",
|
||||
Name: "checkpoint_count",
|
||||
Help: "Number of checkpoint operations performed",
|
||||
Name: "litestream_checkpoint_count",
|
||||
Help: "Number of checkpoint operations performed",
|
||||
}, []string{"db", "mode"})
|
||||
|
||||
checkpointErrorNCounterVec = promauto.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: "litestream",
|
||||
Subsystem: "db",
|
||||
Name: "checkpoint_error_count",
|
||||
Help: "Number of checkpoint errors that have occurred",
|
||||
Name: "litestream_checkpoint_error_count",
|
||||
Help: "Number of checkpoint errors that have occurred",
|
||||
}, []string{"db", "mode"})
|
||||
|
||||
checkpointSecondsCounterVec = promauto.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: "litestream",
|
||||
Subsystem: "db",
|
||||
Name: "checkpoint_seconds",
|
||||
Help: "Time spent checkpointing WAL, in seconds",
|
||||
Name: "litestream_checkpoint_seconds",
|
||||
Help: "Time spent checkpointing WAL, in seconds",
|
||||
}, []string{"db", "mode"})
|
||||
)
|
||||
|
||||
|
||||
Reference in New Issue
Block a user