ed14061578
* 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
455 lines
10 KiB
Go
455 lines
10 KiB
Go
package raft
|
|
|
|
import (
|
|
"context"
|
|
"crypto/md5"
|
|
"encoding/base64"
|
|
fmt "fmt"
|
|
"io"
|
|
"io/ioutil"
|
|
"os"
|
|
"path/filepath"
|
|
"testing"
|
|
"time"
|
|
|
|
"github.com/go-test/deep"
|
|
"github.com/golang/protobuf/proto"
|
|
hclog "github.com/hashicorp/go-hclog"
|
|
uuid "github.com/hashicorp/go-uuid"
|
|
"github.com/hashicorp/raft"
|
|
"github.com/hashicorp/vault/sdk/helper/jsonutil"
|
|
"github.com/hashicorp/vault/sdk/physical"
|
|
bolt "go.etcd.io/bbolt"
|
|
)
|
|
|
|
func getRaft(t testing.TB, bootstrap bool, noStoreState bool) (*RaftBackend, string) {
|
|
raftDir, err := ioutil.TempDir("", "vault-raft-")
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
t.Logf("raft dir: %s", raftDir)
|
|
|
|
logger := hclog.New(&hclog.LoggerOptions{
|
|
Name: "raft",
|
|
Level: hclog.Trace,
|
|
})
|
|
logger.Info("raft dir", "dir", raftDir)
|
|
|
|
conf := map[string]string{
|
|
"path": raftDir,
|
|
"trailing_logs": "100",
|
|
}
|
|
|
|
if noStoreState {
|
|
conf["doNotStoreLatestState"] = ""
|
|
}
|
|
|
|
backendRaw, err := NewRaftBackend(conf, logger)
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
backend := backendRaw.(*RaftBackend)
|
|
|
|
if bootstrap {
|
|
err = backend.Bootstrap(context.Background(), []Peer{Peer{ID: backend.NodeID(), Address: backend.NodeID()}})
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
|
|
err = backend.SetupCluster(context.Background(), nil, nil)
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
|
|
}
|
|
|
|
return backend, raftDir
|
|
}
|
|
|
|
func compareFSMs(t *testing.T, fsm1, fsm2 *FSM) {
|
|
t.Helper()
|
|
index1, config1 := fsm1.LatestState()
|
|
index2, config2 := fsm2.LatestState()
|
|
|
|
if !proto.Equal(index1, index2) {
|
|
t.Fatalf("indexes did not match: %+v != %+v", index1, index2)
|
|
}
|
|
if !proto.Equal(config1, config2) {
|
|
t.Fatalf("configs did not match: %+v != %+v", config1, config2)
|
|
}
|
|
|
|
compareDBs(t, fsm1.db, fsm2.db)
|
|
}
|
|
|
|
func compareDBs(t *testing.T, boltDB1, boltDB2 *bolt.DB) {
|
|
db1 := make(map[string]string)
|
|
db2 := make(map[string]string)
|
|
|
|
err := boltDB1.View(func(tx *bolt.Tx) error {
|
|
|
|
c := tx.Cursor()
|
|
for bucketName, _ := c.First(); bucketName != nil; bucketName, _ = c.Next() {
|
|
b := tx.Bucket(bucketName)
|
|
|
|
cBucket := b.Cursor()
|
|
|
|
for k, v := cBucket.First(); k != nil; k, v = cBucket.Next() {
|
|
db1[string(k)] = base64.StdEncoding.EncodeToString(v)
|
|
}
|
|
}
|
|
|
|
return nil
|
|
})
|
|
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
|
|
err = boltDB2.View(func(tx *bolt.Tx) error {
|
|
c := tx.Cursor()
|
|
for bucketName, _ := c.First(); bucketName != nil; bucketName, _ = c.Next() {
|
|
b := tx.Bucket(bucketName)
|
|
|
|
c := b.Cursor()
|
|
|
|
for k, v := c.First(); k != nil; k, v = c.Next() {
|
|
db2[string(k)] = base64.StdEncoding.EncodeToString(v)
|
|
}
|
|
}
|
|
|
|
return nil
|
|
})
|
|
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
|
|
if diff := deep.Equal(db1, db2); diff != nil {
|
|
t.Fatal(diff)
|
|
}
|
|
}
|
|
|
|
func TestRaft_Backend(t *testing.T) {
|
|
b, dir := getRaft(t, true, true)
|
|
defer os.RemoveAll(dir)
|
|
|
|
physical.ExerciseBackend(t, b)
|
|
}
|
|
|
|
func TestRaft_Backend_ListPrefix(t *testing.T) {
|
|
b, dir := getRaft(t, true, true)
|
|
defer os.RemoveAll(dir)
|
|
|
|
physical.ExerciseBackend_ListPrefix(t, b)
|
|
}
|
|
|
|
func TestRaft_TransactionalBackend(t *testing.T) {
|
|
b, dir := getRaft(t, true, true)
|
|
defer os.RemoveAll(dir)
|
|
|
|
physical.ExerciseTransactionalBackend(t, b)
|
|
}
|
|
|
|
func TestRaft_HABackend(t *testing.T) {
|
|
t.Skip()
|
|
raft, dir := getRaft(t, true, true)
|
|
defer os.RemoveAll(dir)
|
|
raft2, dir2 := getRaft(t, false, true)
|
|
defer os.RemoveAll(dir2)
|
|
|
|
// Add raft2 to the cluster
|
|
addPeer(t, raft, raft2)
|
|
|
|
physical.ExerciseHABackend(t, raft, raft2)
|
|
}
|
|
|
|
func TestRaft_Backend_ThreeNode(t *testing.T) {
|
|
raft1, dir := getRaft(t, true, true)
|
|
raft2, dir2 := getRaft(t, false, true)
|
|
raft3, dir3 := getRaft(t, false, true)
|
|
defer os.RemoveAll(dir)
|
|
defer os.RemoveAll(dir2)
|
|
defer os.RemoveAll(dir3)
|
|
|
|
// Add raft2 to the cluster
|
|
addPeer(t, raft1, raft2)
|
|
|
|
// Add raft3 to the cluster
|
|
addPeer(t, raft1, raft3)
|
|
|
|
physical.ExerciseBackend(t, raft1)
|
|
|
|
time.Sleep(10 * time.Second)
|
|
// Make sure all stores are the same
|
|
compareFSMs(t, raft1.fsm, raft2.fsm)
|
|
compareFSMs(t, raft1.fsm, raft3.fsm)
|
|
}
|
|
|
|
func TestRaft_Recovery(t *testing.T) {
|
|
// Create 4 raft nodes
|
|
raft1, dir1 := getRaft(t, true, true)
|
|
raft2, dir2 := getRaft(t, false, true)
|
|
raft3, dir3 := getRaft(t, false, true)
|
|
raft4, dir4 := getRaft(t, false, true)
|
|
defer os.RemoveAll(dir1)
|
|
defer os.RemoveAll(dir2)
|
|
defer os.RemoveAll(dir3)
|
|
defer os.RemoveAll(dir4)
|
|
|
|
// Add them all to the cluster
|
|
addPeer(t, raft1, raft2)
|
|
addPeer(t, raft1, raft3)
|
|
addPeer(t, raft1, raft4)
|
|
|
|
// Add some data into the FSM
|
|
physical.ExerciseBackend(t, raft1)
|
|
|
|
time.Sleep(10 * time.Second)
|
|
|
|
// Bring down all nodes
|
|
raft1.TeardownCluster(nil)
|
|
raft2.TeardownCluster(nil)
|
|
raft3.TeardownCluster(nil)
|
|
raft4.TeardownCluster(nil)
|
|
|
|
// Prepare peers.json
|
|
type RecoveryPeer struct {
|
|
ID string `json:"id"`
|
|
Address string `json:"address"`
|
|
NonVoter bool `json: non_voter`
|
|
}
|
|
|
|
// Leave out node 1 during recovery
|
|
peersList := make([]*RecoveryPeer, 0, 3)
|
|
peersList = append(peersList, &RecoveryPeer{
|
|
ID: raft1.NodeID(),
|
|
Address: raft1.NodeID(),
|
|
NonVoter: false,
|
|
})
|
|
peersList = append(peersList, &RecoveryPeer{
|
|
ID: raft2.NodeID(),
|
|
Address: raft2.NodeID(),
|
|
NonVoter: false,
|
|
})
|
|
peersList = append(peersList, &RecoveryPeer{
|
|
ID: raft4.NodeID(),
|
|
Address: raft4.NodeID(),
|
|
NonVoter: false,
|
|
})
|
|
|
|
peersJSONBytes, err := jsonutil.EncodeJSON(peersList)
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
err = ioutil.WriteFile(filepath.Join(filepath.Join(dir1, raftState), "peers.json"), peersJSONBytes, 0644)
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
err = ioutil.WriteFile(filepath.Join(filepath.Join(dir2, raftState), "peers.json"), peersJSONBytes, 0644)
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
err = ioutil.WriteFile(filepath.Join(filepath.Join(dir4, raftState), "peers.json"), peersJSONBytes, 0644)
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
|
|
// Bring up the nodes again
|
|
raft1.SetupCluster(context.Background(), nil, nil)
|
|
raft2.SetupCluster(context.Background(), nil, nil)
|
|
raft4.SetupCluster(context.Background(), nil, nil)
|
|
|
|
peers, err := raft1.Peers(context.Background())
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
if len(peers) != 3 {
|
|
t.Fatalf("failed to recover the cluster")
|
|
}
|
|
|
|
time.Sleep(10 * time.Second)
|
|
|
|
compareFSMs(t, raft1.fsm, raft2.fsm)
|
|
compareFSMs(t, raft1.fsm, raft4.fsm)
|
|
}
|
|
|
|
func TestRaft_TransactionalBackend_ThreeNode(t *testing.T) {
|
|
raft1, dir := getRaft(t, true, true)
|
|
raft2, dir2 := getRaft(t, false, true)
|
|
raft3, dir3 := getRaft(t, false, true)
|
|
defer os.RemoveAll(dir)
|
|
defer os.RemoveAll(dir2)
|
|
defer os.RemoveAll(dir3)
|
|
|
|
// Add raft2 to the cluster
|
|
addPeer(t, raft1, raft2)
|
|
|
|
// Add raft3 to the cluster
|
|
addPeer(t, raft1, raft3)
|
|
|
|
physical.ExerciseTransactionalBackend(t, raft1)
|
|
|
|
time.Sleep(10 * time.Second)
|
|
// Make sure all stores are the same
|
|
compareFSMs(t, raft1.fsm, raft2.fsm)
|
|
compareFSMs(t, raft1.fsm, raft3.fsm)
|
|
}
|
|
|
|
func TestRaft_Backend_MaxSize(t *testing.T) {
|
|
// Set the max size a little lower for the test
|
|
maxCommandSizeBytes = 10 * 1024
|
|
|
|
b, dir := getRaft(t, true, true)
|
|
defer os.RemoveAll(dir)
|
|
|
|
// Test a value slightly below the max size
|
|
value := make([]byte, maxCommandSizeBytes-100)
|
|
err := b.Put(context.Background(), &physical.Entry{
|
|
Key: "key",
|
|
Value: value,
|
|
})
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
|
|
// Test value at max size, should error
|
|
value = make([]byte, maxCommandSizeBytes)
|
|
err = b.Put(context.Background(), &physical.Entry{
|
|
Key: "key",
|
|
Value: value,
|
|
})
|
|
if err != ErrCommandTooLarge {
|
|
t.Fatal(err)
|
|
}
|
|
}
|
|
|
|
func TestRaft_Backend_Performance(t *testing.T) {
|
|
b, dir := getRaft(t, true, false)
|
|
defer os.RemoveAll(dir)
|
|
|
|
defaultConfig := raft.DefaultConfig()
|
|
|
|
localConfig := raft.DefaultConfig()
|
|
b.applyConfigSettings(localConfig)
|
|
|
|
if localConfig.ElectionTimeout != defaultConfig.ElectionTimeout*5 {
|
|
t.Fatalf("bad config: %v", localConfig)
|
|
}
|
|
if localConfig.HeartbeatTimeout != defaultConfig.HeartbeatTimeout*5 {
|
|
t.Fatalf("bad config: %v", localConfig)
|
|
}
|
|
if localConfig.LeaderLeaseTimeout != defaultConfig.LeaderLeaseTimeout*5 {
|
|
t.Fatalf("bad config: %v", localConfig)
|
|
}
|
|
|
|
b.conf = map[string]string{
|
|
"path": dir,
|
|
"performance_multiplier": "5",
|
|
}
|
|
|
|
localConfig = raft.DefaultConfig()
|
|
b.applyConfigSettings(localConfig)
|
|
|
|
if localConfig.ElectionTimeout != defaultConfig.ElectionTimeout*5 {
|
|
t.Fatalf("bad config: %v", localConfig)
|
|
}
|
|
if localConfig.HeartbeatTimeout != defaultConfig.HeartbeatTimeout*5 {
|
|
t.Fatalf("bad config: %v", localConfig)
|
|
}
|
|
if localConfig.LeaderLeaseTimeout != defaultConfig.LeaderLeaseTimeout*5 {
|
|
t.Fatalf("bad config: %v", localConfig)
|
|
}
|
|
|
|
b.conf = map[string]string{
|
|
"path": dir,
|
|
"performance_multiplier": "1",
|
|
}
|
|
|
|
localConfig = raft.DefaultConfig()
|
|
b.applyConfigSettings(localConfig)
|
|
|
|
if localConfig.ElectionTimeout != defaultConfig.ElectionTimeout {
|
|
t.Fatalf("bad config: %v", localConfig)
|
|
}
|
|
if localConfig.HeartbeatTimeout != defaultConfig.HeartbeatTimeout {
|
|
t.Fatalf("bad config: %v", localConfig)
|
|
}
|
|
if localConfig.LeaderLeaseTimeout != defaultConfig.LeaderLeaseTimeout {
|
|
t.Fatalf("bad config: %v", localConfig)
|
|
}
|
|
|
|
}
|
|
|
|
func BenchmarkDB_Puts(b *testing.B) {
|
|
raft, dir := getRaft(b, true, false)
|
|
defer os.RemoveAll(dir)
|
|
raft2, dir2 := getRaft(b, true, false)
|
|
defer os.RemoveAll(dir2)
|
|
|
|
bench := func(b *testing.B, s physical.Backend, dataSize int) {
|
|
data, err := uuid.GenerateRandomBytes(dataSize)
|
|
if err != nil {
|
|
b.Fatal(err)
|
|
}
|
|
|
|
ctx := context.Background()
|
|
pe := &physical.Entry{
|
|
Value: data,
|
|
}
|
|
testName := b.Name()
|
|
|
|
b.ResetTimer()
|
|
for i := 0; i < b.N; i++ {
|
|
pe.Key = fmt.Sprintf("%x", md5.Sum([]byte(fmt.Sprintf("%s-%d", testName, i))))
|
|
err := s.Put(ctx, pe)
|
|
if err != nil {
|
|
b.Fatal(err)
|
|
}
|
|
}
|
|
}
|
|
|
|
b.Run("256b", func(b *testing.B) { bench(b, raft, 256) })
|
|
b.Run("256kb", func(b *testing.B) { bench(b, raft2, 256*1024) })
|
|
}
|
|
|
|
func BenchmarkDB_Snapshot(b *testing.B) {
|
|
raft, dir := getRaft(b, true, false)
|
|
defer os.RemoveAll(dir)
|
|
|
|
data, err := uuid.GenerateRandomBytes(256 * 1024)
|
|
if err != nil {
|
|
b.Fatal(err)
|
|
}
|
|
|
|
ctx := context.Background()
|
|
pe := &physical.Entry{
|
|
Value: data,
|
|
}
|
|
testName := b.Name()
|
|
|
|
for i := 0; i < 100; i++ {
|
|
pe.Key = fmt.Sprintf("%x", md5.Sum([]byte(fmt.Sprintf("%s-%d", testName, i))))
|
|
err = raft.Put(ctx, pe)
|
|
if err != nil {
|
|
b.Fatal(err)
|
|
}
|
|
}
|
|
|
|
bench := func(b *testing.B, s *FSM) {
|
|
b.ResetTimer()
|
|
for i := 0; i < b.N; i++ {
|
|
pe.Key = fmt.Sprintf("%x", md5.Sum([]byte(fmt.Sprintf("%s-%d", testName, i))))
|
|
s.writeTo(ctx, discardCloser{Writer: ioutil.Discard}, discardCloser{Writer: ioutil.Discard})
|
|
}
|
|
}
|
|
|
|
b.Run("256kb", func(b *testing.B) { bench(b, raft.fsm) })
|
|
}
|
|
|
|
type discardCloser struct {
|
|
io.Writer
|
|
}
|
|
|
|
func (d discardCloser) Close() error { return nil }
|
|
func (d discardCloser) CloseWithError(error) error { return nil }
|