open-vault/vault/init.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

389 lines
11 KiB
Go

package vault
import (
"context"
"encoding/base64"
"encoding/hex"
"errors"
"fmt"
"net/url"
"sync/atomic"
"github.com/hashicorp/vault/physical/raft"
"github.com/hashicorp/vault/sdk/logical"
"github.com/hashicorp/errwrap"
"github.com/hashicorp/vault/helper/namespace"
"github.com/hashicorp/vault/helper/pgpkeys"
"github.com/hashicorp/vault/shamir"
)
// InitParams keeps the init function from being littered with too many
// params, that's it!
type InitParams struct {
BarrierConfig *SealConfig
RecoveryConfig *SealConfig
RootTokenPGPKey string
}
// InitResult is used to provide the key parts back after
// they are generated as part of the initialization.
type InitResult struct {
SecretShares [][]byte
RecoveryShares [][]byte
RootToken string
}
var (
initPTFunc = func(c *Core) func() { return nil }
initInProgress uint32
)
// Initialized checks if the Vault is already initialized
func (c *Core) Initialized(ctx context.Context) (bool, error) {
// Check the barrier first
init, err := c.barrier.Initialized(ctx)
if err != nil {
c.logger.Error("barrier init check failed", "error", err)
return false, err
}
if !init {
c.logger.Info("security barrier not initialized")
return false, nil
}
// Verify the seal configuration
sealConf, err := c.seal.BarrierConfig(ctx)
if err != nil {
return false, err
}
if sealConf == nil {
return false, fmt.Errorf("core: barrier reports initialized but no seal configuration found")
}
return true, nil
}
func (c *Core) generateShares(sc *SealConfig) ([]byte, [][]byte, error) {
// Generate a master key
masterKey, err := c.barrier.GenerateKey()
if err != nil {
return nil, nil, errwrap.Wrapf("key generation failed: {{err}}", err)
}
// Return the master key if only a single key part is used
var unsealKeys [][]byte
if sc.SecretShares == 1 {
unsealKeys = append(unsealKeys, masterKey)
} else {
// Split the master key using the Shamir algorithm
shares, err := shamir.Split(masterKey, sc.SecretShares, sc.SecretThreshold)
if err != nil {
return nil, nil, errwrap.Wrapf("failed to generate barrier shares: {{err}}", err)
}
unsealKeys = shares
}
// If we have PGP keys, perform the encryption
if len(sc.PGPKeys) > 0 {
hexEncodedShares := make([][]byte, len(unsealKeys))
for i, _ := range unsealKeys {
hexEncodedShares[i] = []byte(hex.EncodeToString(unsealKeys[i]))
}
_, encryptedShares, err := pgpkeys.EncryptShares(hexEncodedShares, sc.PGPKeys)
if err != nil {
return nil, nil, err
}
unsealKeys = encryptedShares
}
return masterKey, unsealKeys, nil
}
// Initialize is used to initialize the Vault with the given
// configurations.
func (c *Core) Initialize(ctx context.Context, initParams *InitParams) (*InitResult, error) {
atomic.StoreUint32(&initInProgress, 1)
defer atomic.StoreUint32(&initInProgress, 0)
barrierConfig := initParams.BarrierConfig
recoveryConfig := initParams.RecoveryConfig
if c.seal.RecoveryKeySupported() {
if recoveryConfig == nil {
return nil, fmt.Errorf("recovery configuration must be supplied")
}
if recoveryConfig.SecretShares < 1 {
return nil, fmt.Errorf("recovery configuration must specify a positive number of shares")
}
// Check if the seal configuration is valid
if err := recoveryConfig.Validate(); err != nil {
c.logger.Error("invalid recovery configuration", "error", err)
return nil, errwrap.Wrapf("invalid recovery configuration: {{err}}", err)
}
}
// Check if the seal configuration is valid
if err := barrierConfig.Validate(); err != nil {
c.logger.Error("invalid seal configuration", "error", err)
return nil, errwrap.Wrapf("invalid seal configuration: {{err}}", err)
}
// Avoid an initialization race
c.stateLock.Lock()
defer c.stateLock.Unlock()
// Check if we are initialized
init, err := c.Initialized(ctx)
if err != nil {
return nil, err
}
if init {
return nil, ErrAlreadyInit
}
// If we have clustered storage, set it up now
if raftStorage, ok := c.underlyingPhysical.(*raft.RaftBackend); ok {
parsedClusterAddr, err := url.Parse(c.clusterAddr)
if err != nil {
return nil, errwrap.Wrapf("error parsing cluster address: {{err}}", err)
}
if err := c.underlyingPhysical.(*raft.RaftBackend).Bootstrap(ctx, []raft.Peer{
{
ID: c.underlyingPhysical.(*raft.RaftBackend).NodeID(),
Address: parsedClusterAddr.Host,
},
}); err != nil {
return nil, errwrap.Wrapf("could not bootstrap clustered storage: {{err}}", err)
}
if err := raftStorage.SetupCluster(ctx, nil, nil); err != nil {
return nil, errwrap.Wrapf("could not start clustered storage: {{err}}", err)
}
defer func() {
if err := raftStorage.TeardownCluster(nil); err != nil {
c.logger.Error("failed to stop raft storage", "error", err)
}
}()
}
err = c.seal.Init(ctx)
if err != nil {
c.logger.Error("failed to initialize seal", "error", err)
return nil, errwrap.Wrapf("error initializing seal: {{err}}", err)
}
barrierKey, barrierUnsealKeys, err := c.generateShares(barrierConfig)
if err != nil {
c.logger.Error("error generating shares", "error", err)
return nil, err
}
initPTCleanup := initPTFunc(c)
if initPTCleanup != nil {
defer initPTCleanup()
}
// Initialize the barrier
if err := c.barrier.Initialize(ctx, barrierKey); err != nil {
c.logger.Error("failed to initialize barrier", "error", err)
return nil, errwrap.Wrapf("failed to initialize barrier: {{err}}", err)
}
if c.logger.IsInfo() {
c.logger.Info("security barrier initialized", "shares", barrierConfig.SecretShares, "threshold", barrierConfig.SecretThreshold)
}
// Unseal the barrier
if err := c.barrier.Unseal(ctx, barrierKey); err != nil {
c.logger.Error("failed to unseal barrier", "error", err)
return nil, errwrap.Wrapf("failed to unseal barrier: {{err}}", err)
}
// Ensure the barrier is re-sealed
defer func() {
// Defers are LIFO so we need to run this here too to ensure the stop
// happens before sealing. preSeal also stops, so we just make the
// stopping safe against multiple calls.
if err := c.barrier.Seal(); err != nil {
c.logger.Error("failed to seal barrier", "error", err)
}
}()
err = c.seal.SetBarrierConfig(ctx, barrierConfig)
if err != nil {
c.logger.Error("failed to save barrier configuration", "error", err)
return nil, errwrap.Wrapf("barrier configuration saving failed: {{err}}", err)
}
// If we are storing shares, pop them out of the returned results and push
// them through the seal
if barrierConfig.StoredShares > 0 {
var keysToStore [][]byte
for i := 0; i < barrierConfig.StoredShares; i++ {
keysToStore = append(keysToStore, barrierUnsealKeys[0])
barrierUnsealKeys = barrierUnsealKeys[1:]
}
if err := c.seal.SetStoredKeys(ctx, keysToStore); err != nil {
c.logger.Error("failed to store keys", "error", err)
return nil, errwrap.Wrapf("failed to store keys: {{err}}", err)
}
}
results := &InitResult{
SecretShares: barrierUnsealKeys,
}
// Perform initial setup
if err := c.setupCluster(ctx); err != nil {
c.logger.Error("cluster setup failed during init", "error", err)
return nil, err
}
// Start tracking
if initPTCleanup != nil {
initPTCleanup()
}
activeCtx, ctxCancel := context.WithCancel(namespace.RootContext(nil))
if err := c.postUnseal(activeCtx, ctxCancel, standardUnsealStrategy{}); err != nil {
c.logger.Error("post-unseal setup failed during init", "error", err)
return nil, err
}
// Save the configuration regardless, but only generate a key if it's not
// disabled. When using recovery keys they are stored in the barrier, so
// this must happen post-unseal.
if c.seal.RecoveryKeySupported() {
err = c.seal.SetRecoveryConfig(ctx, recoveryConfig)
if err != nil {
c.logger.Error("failed to save recovery configuration", "error", err)
return nil, errwrap.Wrapf("recovery configuration saving failed: {{err}}", err)
}
if recoveryConfig.SecretShares > 0 {
recoveryKey, recoveryUnsealKeys, err := c.generateShares(recoveryConfig)
if err != nil {
c.logger.Error("failed to generate recovery shares", "error", err)
return nil, err
}
err = c.seal.SetRecoveryKey(ctx, recoveryKey)
if err != nil {
return nil, err
}
results.RecoveryShares = recoveryUnsealKeys
}
}
// Generate a new root token
rootToken, err := c.tokenStore.rootToken(ctx)
if err != nil {
c.logger.Error("root token generation failed", "error", err)
return nil, err
}
results.RootToken = rootToken.ID
c.logger.Info("root token generated")
if initParams.RootTokenPGPKey != "" {
_, encryptedVals, err := pgpkeys.EncryptShares([][]byte{[]byte(results.RootToken)}, []string{initParams.RootTokenPGPKey})
if err != nil {
c.logger.Error("root token encryption failed", "error", err)
return nil, err
}
results.RootToken = base64.StdEncoding.EncodeToString(encryptedVals[0])
}
if _, ok := c.underlyingPhysical.(*raft.RaftBackend); ok {
raftTLS, err := raft.GenerateTLSKey()
if err != nil {
return nil, err
}
keyring := &raft.RaftTLSKeyring{
Keys: []*raft.RaftTLSKey{raftTLS},
ActiveKeyID: raftTLS.ID,
}
entry, err := logical.StorageEntryJSON(raftTLSStoragePath, keyring)
if err != nil {
return nil, err
}
if err := c.barrier.Put(ctx, entry); err != nil {
return nil, err
}
}
// Prepare to re-seal
if err := c.preSeal(); err != nil {
c.logger.Error("pre-seal teardown failed", "error", err)
return nil, err
}
return results, nil
}
// UnsealWithStoredKeys performs auto-unseal using stored keys. An error
// return value of "nil" implies the Vault instance is unsealed.
//
// Callers should attempt to retry any NonFatalErrors. Callers should
// not re-attempt fatal errors.
func (c *Core) UnsealWithStoredKeys(ctx context.Context) error {
c.unsealWithStoredKeysLock.Lock()
defer c.unsealWithStoredKeysLock.Unlock()
if !c.seal.StoredKeysSupported() {
return nil
}
// Disallow auto-unsealing when migrating
if c.IsInSealMigration() {
return NewNonFatalError(errors.New("cannot auto-unseal during seal migration"))
}
sealed := c.Sealed()
if !sealed {
c.Logger().Warn("attempted unseal with stored keys, but vault is already unsealed")
return nil
}
c.Logger().Info("stored unseal keys supported, attempting fetch")
keys, err := c.seal.GetStoredKeys(ctx)
if err != nil {
return NewNonFatalError(errwrap.Wrapf("fetching stored unseal keys failed: {{err}}", err))
}
// This usually happens when auto-unseal is configured, but the servers have
// not been initialized yet.
if len(keys) == 0 {
return NewNonFatalError(errors.New("stored unseal keys are supported, but none were found"))
}
unsealed := false
keysUsed := 0
for _, key := range keys {
unsealed, err = c.Unseal(key)
if err != nil {
return NewNonFatalError(errwrap.Wrapf("unseal with stored key failed: {{err}}", err))
}
keysUsed++
if unsealed {
break
}
}
if !unsealed {
// This most likely means that the user configured Vault to only store a
// subset of the required threshold of keys. We still consider this a
// "success", since trying again would yield the same result.
c.Logger().Warn("vault still sealed after using stored unseal keys", "stored_keys_used", keysUsed)
} else {
c.Logger().Info("unsealed with stored keys", "stored_keys_used", keysUsed)
}
return nil
}