Validate sqlite header

This commit is contained in:
Ben Johnson
2020-11-06 12:33:01 -07:00
parent 9a2a30f239
commit d964e4199a
8 changed files with 235 additions and 149 deletions

105
db.go
View File

@@ -2,10 +2,14 @@ package litestream
import ( import (
"context" "context"
"io"
"log"
"os" "os"
"path/filepath" "path/filepath"
"strings" "strings"
"sync" "sync"
"github.com/benbjohnson/litestream/sqlite"
) )
const ( const (
@@ -18,12 +22,23 @@ const (
// DB represents an instance of a managed SQLite database in the file system. // DB represents an instance of a managed SQLite database in the file system.
type DB struct { type DB struct {
mu sync.Mutex
path string path string
inTx bool // currently in transaction
isHeaderValid bool // true if meta page contains SQLITE3 header
isWALEnabled bool // true if file format version specifies WAL
// Tracks offset of WAL data.
processedWALByteN int64 // bytes copied to shadow WAL
pendingWALByteN int64 // bytes pending copy to shadow WAL
ctx context.Context ctx context.Context
cancel func() cancel func()
wg sync.WaitGroup wg sync.WaitGroup
// Database-specific logger
logFile *os.File
logger *log.Logger
} }
// NewDB returns a new instance of DB for a given path. // NewDB returns a new instance of DB for a given path.
@@ -38,39 +53,107 @@ func (db *DB) Path() string {
return db.path return db.path
} }
// MetaPath returns the path to the database metadata. // InternalMetaPath returns the path to the database metadata.
func (db *DB) MetaPath() string { func (db *DB) InternalMetaPath() string {
dir, file := filepath.Split(db.path) dir, file := filepath.Split(db.path)
return filepath.Join(dir, "."+file+MetaDirSuffix) return filepath.Join(db.node.fs.TargetPath, dir, "."+file+MetaDirSuffix)
} }
// WALPath returns the path to the internal WAL directory. // InternalWALPath returns the path to the internal WAL directory.
func (db *DB) WALPath() string { func (db *DB) InternalWALPath() string {
return filepath.Join(db.MetaPath(), WALDirName) return filepath.Join(db.MetaPath(), WALDirName)
} }
// LogPath returns the path to the internal log directory. // InternalLogPath returns the path to the internal log directory.
func (db *DB) LogPath() string { func (db *DB) InternalLogPath() string {
return filepath.Join(db.MetaPath(), LogFilename) return filepath.Join(db.MetaPath(), LogFilename)
} }
// Open loads the configuration file // Open loads the configuration file
func (db *DB) Open() error { func (db *DB) Open() (err error) {
// Ensure meta directory exists. db.mu.Lock()
defer db.mu.Unlock()
// Ensure meta directory structure exists.
if err := os.MkdirAll(db.MetaPath(), 0600); err != nil { if err := os.MkdirAll(db.MetaPath(), 0600); err != nil {
return err return err
} else if err := os.MkdirAll(db.WALPath(), 0600); err != nil {
return err
}
// Initialize per-db logger.
if db.logFile, err = os.OpenFile(db.LogPath(), os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0600); err != nil {
return err
}
db.logger = log.New(db.logFile, "", log.LstdFlags)
// If database file exists, read & set the header.
if err := db.readHeader(); err != nil {
db.setHeader(nil) // invalidate header
db.logger.Printf("cannot read db header: %s", err)
} }
return nil return nil
} }
// Close stops management of the database. // Close stops management of the database.
func (db *DB) Close() error { func (db *DB) Close() (err error) {
db.cancel() db.cancel()
db.wg.Wait() db.wg.Wait()
// Close per-db log file.
if e := db.logFile.Close(); e != nil && err == nil {
err = e
}
return err
}
// readHeader reads the SQLite header and sets the initial DB flags.
func (db *DB) readHeader() error {
f, err := os.Open(db.path)
if err != nil {
return err
}
defer f.Close()
hdr := make([]byte, sqlite.HeaderSize)
if _, err := io.ReadFull(f, hdr); err != nil {
return err
}
db.setHeader(hdr)
return nil return nil
} }
// Valid returns true if there is a valid, WAL-enabled SQLite database on-disk.
func (db *DB) Valid() bool {
db.mu.Lock()
defer db.mu.Unlock()
return db.valid()
}
func (db *DB) valid() bool {
return db.isHeaderValid && db.isWALEnabled
}
// SetHeader checks if the page has a valid header & uses a WAL.
func (db *DB) SetHeader(page []byte) {
db.mu.Lock()
defer db.mu.Unlock()
db.setHeader(page)
}
func (db *DB) setHeader(page []byte) {
db.isHeaderValid = sqlite.IsValidHeader(page)
db.isWALEnabled = sqlite.IsWALEnabled(page)
}
func (db *DB) AddPendingWALByteN(n int64) {
db.mu.Lock()
defer db.mu.Unlock()
db.pendingWALByteN += n
}
// IsMetaDir returns true if base in path is hidden and ends in "-litestream". // IsMetaDir returns true if base in path is hidden and ends in "-litestream".
func IsMetaDir(path string) bool { func IsMetaDir(path string) bool {
base := filepath.Base(path) base := filepath.Base(path)

View File

@@ -17,11 +17,11 @@ dir/
.db-litestream/ .db-litestream/
log # recent event log log # recent event log
stat # per-db Prometheus statistics stat # per-db Prometheus statistics
snapshot # stores snapshot number (e.g. 0000000000000001) generation # current generation number
wal/ # each WAL file contains pages in flush interval wal/ # each WAL file contains pages in flush interval
active # active WAL file exists until flush; renamed active.wal # active WAL file exists until flush; renamed
0000000000000001.wal.gz # flushed, compressed WAL files 000000000000001.wal.gz # flushed, compressed WAL files
0000000000000002.wal.gz 000000000000002.wal.gz
``` ```
### Remote (S3) ### Remote (S3)
@@ -29,13 +29,22 @@ dir/
``` ```
bkt/ bkt/
db/ # database path db/ # database path
0000000000000001/ # snapshot directory 00000001/ # snapshot directory
snapshot # full db snapshot snapshot # full db snapshot
0000000000000001.wal.gz # compressed WAL file 000000000000001.wal.gz # compressed WAL file
0000000000000002.wal.gz 000000000000002.wal.gz
0000000000000002/ 00000002/
snapshot snapshot/
0000000000000001-0000000000000003.tar.gz 000000000000000.snapshot
scheduled/
daily/
20000101T000000Z-000000000000023.snapshot
20000102T000000Z-000000000000036.snapshot
monthly/
20000101T000000Z-000000000000023.snapshot
wal/
000000000000001.wal.gz
``` ```

View File

@@ -69,6 +69,13 @@ func (f *FileSystem) Open() error {
}) })
} }
// DB returns the DB object associated with path.
func (f *FileSystem) DB(path string) *DB {
f.mu.RLock()
defer f.mu.RUnlock()
return f.dbs[path]
}
// OpenDB initializes a DB for a given path. // OpenDB initializes a DB for a given path.
func (f *FileSystem) OpenDB(path string) error { func (f *FileSystem) OpenDB(path string) error {
f.mu.Lock() f.mu.Lock()

111
handle.go
View File

@@ -10,6 +10,7 @@ import (
"bazil.org/fuse" "bazil.org/fuse"
"bazil.org/fuse/fs" "bazil.org/fuse/fs"
"github.com/benbjohnson/litestream/sqlite"
) )
var _ fs.HandleFlusher = (*Handle)(nil) var _ fs.HandleFlusher = (*Handle)(nil)
@@ -25,9 +26,15 @@ var _ fs.HandleWriter = (*Handle)(nil)
// Handle represents a FUSE file handle. // Handle represents a FUSE file handle.
type Handle struct { type Handle struct {
node *Node
f *os.File f *os.File
} }
// NewHandle returns a new instance of Handle.
func NewHandle(n *Node, f *os.File) *Handle {
return &Handle{node: n, f: f}
}
// Release closes the underlying file descriptor. // Release closes the underlying file descriptor.
func (h *Handle) Release(ctx context.Context, req *fuse.ReleaseRequest) (err error) { func (h *Handle) Release(ctx context.Context, req *fuse.ReleaseRequest) (err error) {
return h.f.Close() return h.f.Close()
@@ -49,10 +56,38 @@ func (h *Handle) Write(ctx context.Context, req *fuse.WriteRequest, resp *fuse.W
log.Printf("write: name=%s offset=%d n=%d", h.f.Name(), req.Offset, len(req.Data)) log.Printf("write: name=%s offset=%d n=%d", h.f.Name(), req.Offset, len(req.Data))
println(HexDump(req.Data)) println(HexDump(req.Data))
resp.Size, err = h.f.WriteAt(req.Data, req.Offset) if resp.Size, err = h.f.WriteAt(req.Data, req.Offset); err != nil {
// TODO: Invalidate node DB state.
return err return err
} }
// Check if handle reference a managed database.
db := h.node.DB()
if db == nil {
return nil
}
// If this is the DB file, update the DB state based on the header.
if !sqlite.IsWALPath(h.node.Path()) {
// TODO: Header write could theoretically occur anywhere in first 100 bytes.
// If updating the header page, first validate it.
if req.Offset == 0 {
db.SetHeader(req.Data)
}
return nil
}
// Ignore if the DB is not in a valid state (header + wal enabled).
if !db.Valid() {
return nil
}
// Otherwise this is the WAL file so we should append the WAL data.
db.AddPendingWALByteN(int64(len(req.Data)))
return nil
}
// Flush is called when a file handle is synced to disk. Implements fs.HandleFlusher. // Flush is called when a file handle is synced to disk. Implements fs.HandleFlusher.
func (h *Handle) Flush(ctx context.Context, req *fuse.FlushRequest) (err error) { func (h *Handle) Flush(ctx context.Context, req *fuse.FlushRequest) (err error) {
return h.f.Sync() return h.f.Sync()
@@ -75,77 +110,3 @@ func (h *Handle) ReadDirAll(ctx context.Context) (ents []fuse.Dirent, err error)
sort.Slice(ents, func(i, j int) bool { return ents[i].Name < ents[j].Name }) sort.Slice(ents, func(i, j int) bool { return ents[i].Name < ents[j].Name })
return ents, nil return ents, nil
} }
/*
// Lock tries to acquire a lock on a byte range of the node. If a
// conflicting lock is already held, returns syscall.EAGAIN.
func (h *Handle) Lock(ctx context.Context, req *fuse.LockRequest) error {
log.Printf("dbg/lock %p %s -- %#v", h, h.f.Name(), req.Lock)
if err := syscall.FcntlFlock(h.f.Fd(), unix.F_OFD_SETLK, &syscall.Flock_t{
Type: int16(req.Lock.Type),
Whence: io.SeekStart,
Start: int64(req.Lock.Start),
Len: int64(req.Lock.End) - int64(req.Lock.Start),
}); err != nil {
return err
}
return nil
}
// LockWait acquires a lock on a byte range of the node, waiting
// until the lock can be obtained (or context is canceled).
func (h *Handle) LockWait(ctx context.Context, req *fuse.LockWaitRequest) error {
log.Printf("dbg/lockwait %p %s -- %#v", h, h.f.Name(), req.Lock)
return syscall.FcntlFlock(h.f.Fd(), unix.F_OFD_SETLKW, &syscall.Flock_t{
Type: int16(req.Lock.Type),
Whence: io.SeekStart,
Start: int64(req.Lock.Start),
Len: int64(req.Lock.End) - int64(req.Lock.Start),
})
}
// Unlock releases the lock on a byte range of the node. Locks can
// be released also implicitly, see HandleFlockLocker and
// HandlePOSIXLocker.
func (h *Handle) Unlock(ctx context.Context, req *fuse.UnlockRequest) error {
log.Printf("dbg/unlk %p %s -- %#v", h, h.f.Name(), req.Lock)
return syscall.FcntlFlock(h.f.Fd(), unix.F_OFD_SETLK, &syscall.Flock_t{
Type: int16(req.Lock.Type),
Whence: io.SeekStart,
Start: int64(req.Lock.Start),
Len: int64(req.Lock.End) - int64(req.Lock.Start),
})
}
// QueryLock returns the current state of locks held for the byte
// range of the node.
//
// See QueryLockRequest for details on how to respond.
//
// To simplify implementing this method, resp.Lock is prefilled to
// have Lock.Type F_UNLCK, and the whole struct should be
// overwritten for in case of conflicting locks.
func (h *Handle) QueryLock(ctx context.Context, req *fuse.QueryLockRequest, resp *fuse.QueryLockResponse) error {
log.Printf("dbg/querylock %p %s -- %#v", h, h.f.Name(), req.Lock)
flock_t := syscall.Flock_t{
Type: int16(req.Lock.Type),
Whence: io.SeekStart,
Start: int64(req.Lock.Start),
Len: int64(req.Lock.End) - int64(req.Lock.Start),
}
if err := syscall.FcntlFlock(h.f.Fd(), unix.F_OFD_GETLK, &flock_t); err != nil {
return err
}
resp.Lock = fuse.FileLock{
Type: fuse.LockType(flock_t.Type),
Start: uint64(flock_t.Start),
End: uint64(flock_t.Start + flock_t.Len),
PID: flock_t.Pid,
}
return nil
}
*/

View File

@@ -1,48 +1,10 @@
package litestream package litestream
import ( import (
"encoding/binary"
"encoding/hex" "encoding/hex"
"io"
"strings" "strings"
) )
// Magic number specified at the beginning of WAL files.
const (
MagicLittleEndian = 0x377f0682
MagicBigEndian = 0x377f0683
)
const (
WriteVersionOffset = 18
ReadVersionOffset = 19
)
// ReadVersion returns the SQLite write & read version.
// Returns 1 for legacy & 2 for WAL.
func ReadVersion(b []byte) (writeVersion, readVersion uint8, err error) {
if len(b) < ReadVersionOffset {
return 0, 0, io.ErrUnexpectedEOF
}
return b[WriteVersionOffset], b[ReadVersionOffset], nil
}
// Checksum computes a running checksum over a byte slice.
func Checksum(bo binary.ByteOrder, s uint64, b []byte) (_ uint64, err error) {
// Ensure byte slice length is divisible by 8.
if len(b)%8 != 0 {
return 0, ErrChecksumMisaligned
}
// Iterate over 8-byte units and compute checksum.
s0, s1 := uint32(s>>32), uint32(s&0xFFFFFFFF)
for i := 0; i < len(b); i += 8 {
s0 += bo.Uint32(b[i:]) + s1
s1 += bo.Uint32(b[i+4:]) + s0
}
return uint64(s0)<<32 | uint64(s1), nil
}
// HexDump returns hexdump output but with duplicate lines removed. // HexDump returns hexdump output but with duplicate lines removed.
func HexDump(b []byte) string { func HexDump(b []byte) string {
const prefixN = len("00000000") const prefixN = len("00000000")

22
node.go
View File

@@ -5,11 +5,14 @@ import (
"io/ioutil" "io/ioutil"
"os" "os"
"path/filepath" "path/filepath"
"strings"
"sync"
"syscall" "syscall"
"time" "time"
"bazil.org/fuse" "bazil.org/fuse"
"bazil.org/fuse/fs" "bazil.org/fuse/fs"
"github.com/benbjohnson/litestream/sqlite"
) )
var _ fs.Node = (*Node)(nil) var _ fs.Node = (*Node)(nil)
@@ -33,6 +36,7 @@ var _ fs.NodeSymlinker = (*Node)(nil)
// Node represents a file or directory in the file system. // Node represents a file or directory in the file system.
type Node struct { type Node struct {
mu sync.RWMutex
fs *FileSystem // base filesystem fs *FileSystem // base filesystem
path string // path within file system path string // path within file system
} }
@@ -41,10 +45,24 @@ func NewNode(fs *FileSystem, path string) *Node {
return &Node{fs: fs, path: path} return &Node{fs: fs, path: path}
} }
// Path returns the path the node was initialized with.
func (n *Node) Path() string {
return n.path
}
func (n *Node) srcpath() string { func (n *Node) srcpath() string {
return filepath.Join(n.fs.TargetPath, n.path) return filepath.Join(n.fs.TargetPath, n.path)
} }
// DB returns the DB object associated with the node, if any.
// If node points to a "-wal" file then the associated DB is returned.
func (n *Node) DB() *DB {
if strings.HasPrefix(n.path, sqlite.WALSuffix) {
return n.fs.DB(strings.TrimSuffix(n.path, sqlite.WALSuffix))
}
return n.fs.DB(n.path)
}
func (n *Node) Attr(ctx context.Context, a *fuse.Attr) (err error) { func (n *Node) Attr(ctx context.Context, a *fuse.Attr) (err error) {
fi, err := os.Stat(n.srcpath()) fi, err := os.Stat(n.srcpath())
if err != nil { if err != nil {
@@ -252,7 +270,7 @@ func (n *Node) Open(ctx context.Context, req *fuse.OpenRequest, resp *fuse.OpenR
if err != nil { if err != nil {
return nil, err return nil, err
} }
return &Handle{f: f}, nil return NewHandle(n, f), nil
} }
// Create creates a new directory entry in the receiver, which must be a directory. // Create creates a new directory entry in the receiver, which must be a directory.
@@ -261,7 +279,7 @@ func (n *Node) Create(ctx context.Context, req *fuse.CreateRequest, resp *fuse.C
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
} }
return NewNode(n.fs, filepath.Join(n.path, req.Name)), &Handle{f: f}, nil return NewNode(n.fs, filepath.Join(n.path, req.Name)), NewHandle(n, f), nil
} }
func (n *Node) Rename(ctx context.Context, req *fuse.RenameRequest, _newDir fs.Node) (err error) { func (n *Node) Rename(ctx context.Context, req *fuse.RenameRequest, _newDir fs.Node) (err error) {

View File

@@ -1,9 +1,11 @@
package litestream package sqlite
import ( import (
"bytes"
"encoding/binary" "encoding/binary"
"errors" "errors"
"io" "io"
"strings"
) )
// TODO: Pages can be written multiple times before 3.11.0 (https://sqlite.org/releaselog/3_11_0.html) // TODO: Pages can be written multiple times before 3.11.0 (https://sqlite.org/releaselog/3_11_0.html)
@@ -20,6 +22,50 @@ var (
ErrChecksumMisaligned = errors.New("checksum input misaligned") ErrChecksumMisaligned = errors.New("checksum input misaligned")
) )
// HeaderSize is the size of a SQLite 3 database header, in bytes.
const HeaderSize = 100
// WALSuffix is the suffix appended to the end of SQLite WAL path names.
const WALSuffix = "-wal"
// Magic number specified at the beginning of WAL files.
const (
MagicLittleEndian = 0x377f0682
MagicBigEndian = 0x377f0683
)
// IsWALPath returns true if path ends with WALSuffix.
func IsWALPath(path string) bool {
return strings.HasSuffix(path, WALSuffix)
}
// IsValidHeader returns true if page contains the standard SQLITE3 header.
func IsValidHeader(page []byte) bool {
return bytes.HasPrefix(page, []byte("SQLite format 3\x00"))
}
// IsWALEnabled returns true if header page has the file format read & write
// version set to 2 (which indicates WAL).
func IsWALEnabled(page []byte) bool {
return len(page) >= 19 && page[18] == 2 && page[19] == 2
}
// Checksum computes a running checksum over a byte slice.
func Checksum(bo binary.ByteOrder, s uint64, b []byte) (_ uint64, err error) {
// Ensure byte slice length is divisible by 8.
if len(b)%8 != 0 {
return 0, ErrChecksumMisaligned
}
// Iterate over 8-byte units and compute checksum.
s0, s1 := uint32(s>>32), uint32(s&0xFFFFFFFF)
for i := 0; i < len(b); i += 8 {
s0 += bo.Uint32(b[i:]) + s1
s1 += bo.Uint32(b[i+4:]) + s0
}
return uint64(s0)<<32 | uint64(s1), nil
}
// WALHeaderSize is the size of the WAL header, in bytes. // WALHeaderSize is the size of the WAL header, in bytes.
const WALHeaderSize = 32 const WALHeaderSize = 32

View File

@@ -1,11 +1,11 @@
package litestream_test package sqlite_test
import ( import (
"io" "io"
"path/filepath" "path/filepath"
"testing" "testing"
"github.com/benbjohnson/litestream" "github.com/benbjohnson/litestream/sqlite"
) )
func TestWALHeader_MarshalTo(t *testing.T) { func TestWALHeader_MarshalTo(t *testing.T) {