open-vault/physical/raft/logstore/bolt_store.go
Brian Kassouf ed14061578
Raft Storage Backend (#6888)
* Work on raft backend

* Add logstore locally

* Add encryptor and unsealable interfaces

* Add clustering support to raft

* Remove client and handler

* Bootstrap raft on init

* Cleanup raft logic a bit

* More raft work

* Work on TLS config

* More work on bootstrapping

* Fix build

* More work on bootstrapping

* More bootstrapping work

* fix build

* Remove consul dep

* Fix build

* merged oss/master into raft-storage

* Work on bootstrapping

* Get bootstrapping to work

* Clean up FMS and node-id

* Update local node ID logic

* Cleanup node-id change

* Work on snapshotting

* Raft: Add remove peer API (#906)

* Add remove peer API

* Add some comments

* Fix existing snapshotting (#909)

* Raft get peers API (#912)

* Read raft configuration

* address review feedback

* Use the Leadership Transfer API to step-down the active node (#918)

* Raft join and unseal using Shamir keys (#917)

* Raft join using shamir

* Store AEAD instead of master key

* Split the raft join process to answer the challenge after a successful unseal

* get the follower to standby state

* Make unseal work

* minor changes

* Some input checks

* reuse the shamir seal access instead of new default seal access

* refactor joinRaftSendAnswer function

* Synchronously send answer in auto-unseal case

* Address review feedback

* Raft snapshots (#910)

* Fix existing snapshotting

* implement the noop snapshotting

* Add comments and switch log libraries

* add some snapshot tests

* add snapshot test file

* add TODO

* More work on raft snapshotting

* progress on the ConfigStore strategy

* Don't use two buckets

* Update the snapshot store logic to hide the file logic

* Add more backend tests

* Cleanup code a bit

* [WIP] Raft recovery (#938)

* Add recovery functionality

* remove fmt.Printfs

* Fix a few fsm bugs

* Add max size value for raft backend (#942)

* Add max size value for raft backend

* Include physical.ErrValueTooLarge in the message

* Raft snapshot Take/Restore API  (#926)

* Inital work on raft snapshot APIs

* Always redirect snapshot install/download requests

* More work on the snapshot APIs

* Cleanup code a bit

* On restore handle special cases

* Use the seal to encrypt the sha sum file

* Add sealer mechanism and fix some bugs

* Call restore while state lock is held

* Send restore cb trigger through raft log

* Make error messages nicer

* Add test helpers

* Add snapshot test

* Add shamir unseal test

* Add more raft snapshot API tests

* Fix locking

* Change working to initalize

* Add underlying raw object to test cluster core

* Move leaderUUID to core

* Add raft TLS rotation logic (#950)

* Add TLS rotation logic

* Cleanup logic a bit

* Add/Remove from follower state on add/remove peer

* add comments

* Update more comments

* Update request_forwarding_service.proto

* Make sure we populate all nodes in the followerstate obj

* Update times

* Apply review feedback

* Add more raft config setting (#947)

* Add performance config setting

* Add more config options and fix tests

* Test Raft Recovery (#944)

* Test raft recovery

* Leave out a node during recovery

* remove unused struct

* Update physical/raft/snapshot_test.go

* Update physical/raft/snapshot_test.go

* fix vendoring

* Switch to new raft interface

* Remove unused files

* Switch a gogo -> proto instance

* Remove unneeded vault dep in go.sum

* Update helper/testhelpers/testhelpers.go

Co-Authored-By: Calvin Leung Huang <cleung2010@gmail.com>

* Update vault/cluster/cluster.go

* track active key within the keyring itself (#6915)

* track active key within the keyring itself

* lookup and store using the active key ID

* update docstring

* minor refactor

* Small text fixes (#6912)

* Update physical/raft/raft.go

Co-Authored-By: Calvin Leung Huang <cleung2010@gmail.com>

* review feedback

* Move raft logical system into separate file

* Update help text a bit

* Enforce cluster addr is set and use it for raft bootstrapping

* Fix tests

* fix http test panic

* Pull in latest raft-snapshot library

* Add comment
2019-06-20 12:14:58 -07:00

272 lines
6.1 KiB
Go

package logstore
import (
"errors"
"github.com/hashicorp/raft"
bolt "go.etcd.io/bbolt"
)
const (
// Permissions to use on the db file. This is only used if the
// database file does not exist and needs to be created.
dbFileMode = 0600
)
var (
// Bucket names we perform transactions in
dbLogs = []byte("logs")
dbConf = []byte("conf")
// An error indicating a given key does not exist
ErrKeyNotFound = errors.New("not found")
)
// BoltStore provides access to BoltDB for Raft to store and retrieve
// log entries. It also provides key/value storage, and can be used as
// a LogStore and StableStore.
type BoltStore struct {
// conn is the underlying handle to the db.
conn *bolt.DB
// The path to the Bolt database file
path string
}
// Options contains all the configuraiton used to open the BoltDB
type Options struct {
// Path is the file path to the BoltDB to use
Path string
// BoltOptions contains any specific BoltDB options you might
// want to specify [e.g. open timeout]
BoltOptions *bolt.Options
// NoSync causes the database to skip fsync calls after each
// write to the log. This is unsafe, so it should be used
// with caution.
NoSync bool
}
// readOnly returns true if the contained bolt options say to open
// the DB in readOnly mode [this can be useful to tools that want
// to examine the log]
func (o *Options) readOnly() bool {
return o != nil && o.BoltOptions != nil && o.BoltOptions.ReadOnly
}
// NewBoltStore takes a file path and returns a connected Raft backend.
func NewBoltStore(path string) (*BoltStore, error) {
return New(Options{Path: path})
}
// New uses the supplied options to open the BoltDB and prepare it for use as a raft backend.
func New(options Options) (*BoltStore, error) {
// Try to connect
handle, err := bolt.Open(options.Path, dbFileMode, options.BoltOptions)
if err != nil {
return nil, err
}
handle.NoSync = options.NoSync
// Create the new store
store := &BoltStore{
conn: handle,
path: options.Path,
}
// If the store was opened read-only, don't try and create buckets
if !options.readOnly() {
// Set up our buckets
if err := store.initialize(); err != nil {
store.Close()
return nil, err
}
}
return store, nil
}
// initialize is used to set up all of the buckets.
func (b *BoltStore) initialize() error {
tx, err := b.conn.Begin(true)
if err != nil {
return err
}
defer tx.Rollback()
// Create all the buckets
if _, err := tx.CreateBucketIfNotExists(dbLogs); err != nil {
return err
}
if _, err := tx.CreateBucketIfNotExists(dbConf); err != nil {
return err
}
return tx.Commit()
}
// Close is used to gracefully close the DB connection.
func (b *BoltStore) Close() error {
return b.conn.Close()
}
// FirstIndex returns the first known index from the Raft log.
func (b *BoltStore) FirstIndex() (uint64, error) {
tx, err := b.conn.Begin(false)
if err != nil {
return 0, err
}
defer tx.Rollback()
curs := tx.Bucket(dbLogs).Cursor()
if first, _ := curs.First(); first == nil {
return 0, nil
} else {
return bytesToUint64(first), nil
}
}
// LastIndex returns the last known index from the Raft log.
func (b *BoltStore) LastIndex() (uint64, error) {
tx, err := b.conn.Begin(false)
if err != nil {
return 0, err
}
defer tx.Rollback()
curs := tx.Bucket(dbLogs).Cursor()
if last, _ := curs.Last(); last == nil {
return 0, nil
} else {
return bytesToUint64(last), nil
}
}
// GetLog is used to retrieve a log from BoltDB at a given index.
func (b *BoltStore) GetLog(idx uint64, log *raft.Log) error {
tx, err := b.conn.Begin(false)
if err != nil {
return err
}
defer tx.Rollback()
bucket := tx.Bucket(dbLogs)
val := bucket.Get(uint64ToBytes(idx))
if val == nil {
return raft.ErrLogNotFound
}
return decodeMsgPack(val, log)
}
// StoreLog is used to store a single raft log
func (b *BoltStore) StoreLog(log *raft.Log) error {
return b.StoreLogs([]*raft.Log{log})
}
// StoreLogs is used to store a set of raft logs
func (b *BoltStore) StoreLogs(logs []*raft.Log) error {
tx, err := b.conn.Begin(true)
if err != nil {
return err
}
defer tx.Rollback()
for _, log := range logs {
key := uint64ToBytes(log.Index)
val, err := encodeMsgPack(log)
if err != nil {
return err
}
bucket := tx.Bucket(dbLogs)
if err := bucket.Put(key, val.Bytes()); err != nil {
return err
}
}
return tx.Commit()
}
// DeleteRange is used to delete logs within a given range inclusively.
func (b *BoltStore) DeleteRange(min, max uint64) error {
minKey := uint64ToBytes(min)
tx, err := b.conn.Begin(true)
if err != nil {
return err
}
defer tx.Rollback()
curs := tx.Bucket(dbLogs).Cursor()
for k, _ := curs.Seek(minKey); k != nil; k, _ = curs.Next() {
// Handle out-of-range log index
if bytesToUint64(k) > max {
break
}
// Delete in-range log index
if err := curs.Delete(); err != nil {
return err
}
}
return tx.Commit()
}
// Set is used to set a key/value set outside of the raft log
func (b *BoltStore) Set(k, v []byte) error {
tx, err := b.conn.Begin(true)
if err != nil {
return err
}
defer tx.Rollback()
bucket := tx.Bucket(dbConf)
if err := bucket.Put(k, v); err != nil {
return err
}
return tx.Commit()
}
// Get is used to retrieve a value from the k/v store by key
func (b *BoltStore) Get(k []byte) ([]byte, error) {
tx, err := b.conn.Begin(false)
if err != nil {
return nil, err
}
defer tx.Rollback()
bucket := tx.Bucket(dbConf)
val := bucket.Get(k)
if val == nil {
return nil, ErrKeyNotFound
}
return append([]byte(nil), val...), nil
}
// SetUint64 is like Set, but handles uint64 values
func (b *BoltStore) SetUint64(key []byte, val uint64) error {
return b.Set(key, uint64ToBytes(val))
}
// GetUint64 is like Get, but handles uint64 values
func (b *BoltStore) GetUint64(key []byte) (uint64, error) {
val, err := b.Get(key)
if err != nil {
return 0, err
}
return bytesToUint64(val), nil
}
// Sync performs an fsync on the database file handle. This is not necessary
// under normal operation unless NoSync is enabled, in which this forces the
// database file to sync against the disk.
func (b *BoltStore) Sync() error {
return b.conn.Sync()
}