1
0
vault-redux/vault/init.go

498 lines
15 KiB
Go
Raw Normal View History

// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: BUSL-1.1
2016-04-04 17:44:22 +03:00
package vault
import (
"context"
"encoding/base64"
2016-04-04 17:44:22 +03:00
"encoding/hex"
"errors"
2016-04-04 17:44:22 +03:00
"fmt"
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 22:14:58 +03:00
"net/url"
2019-02-19 23:03:02 +03:00
"sync/atomic"
2016-04-04 17:44:22 +03:00
wrapping "github.com/hashicorp/go-kms-wrapping/v2"
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 22:14:58 +03:00
"github.com/hashicorp/vault/physical/raft"
"github.com/hashicorp/vault/vault/seal"
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 22:14:58 +03:00
2018-09-18 06:03:00 +03:00
"github.com/hashicorp/vault/helper/namespace"
2016-04-04 17:44:22 +03:00
"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
// LegacyShamirSeal should only be used in test code, we don't want to
// give the user a way to create legacy shamir seals.
LegacyShamirSeal bool
}
2016-04-04 17:44:22 +03:00
// 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
}
2018-09-18 06:03:00 +03:00
var (
initPTFunc = func(c *Core) func() { return nil }
initInProgress uint32
ErrInitWithoutAutoloading = errors.New("cannot initialize storage without an autoloaded license")
2018-09-18 06:03:00 +03:00
)
Recovery Mode (#7559) * Initial work * rework * s/dr/recovery * Add sys/raw support to recovery mode (#7577) * Factor the raw paths out so they can be run with a SystemBackend. # Conflicts: # vault/logical_system.go * Add handleLogicalRecovery which is like handleLogical but is only sufficient for use with the sys-raw endpoint in recovery mode. No authentication is done yet. * Integrate with recovery-mode. We now handle unauthenticated sys/raw requests, albeit on path v1/raw instead v1/sys/raw. * Use sys/raw instead raw during recovery. * Don't bother persisting the recovery token. Authenticate sys/raw requests with it. * RecoveryMode: Support generate-root for autounseals (#7591) * Recovery: Abstract config creation and log settings * Recovery mode integration test. (#7600) * Recovery: Touch up (#7607) * Recovery: Touch up * revert the raw backend creation changes * Added recovery operation token prefix * Move RawBackend to its own file * Update API path and hit it using CLI flag on generate-root * Fix a panic triggered when handling a request that yields a nil response. (#7618) * Improve integ test to actually make changes while in recovery mode and verify they're still there after coming back in regular mode. * Refuse to allow a second recovery token to be generated. * Resize raft cluster to size 1 and start as leader (#7626) * RecoveryMode: Setup raft cluster post unseal (#7635) * Setup raft cluster post unseal in recovery mode * Remove marking as unsealed as its not needed * Address review comments * Accept only one seal config in recovery mode as there is no scope for migration
2019-10-15 07:55:31 +03:00
func (c *Core) InitializeRecovery(ctx context.Context) error {
if !c.recoveryMode {
return nil
}
raftStorage, ok := c.underlyingPhysical.(*raft.RaftBackend)
if !ok {
return nil
}
parsedClusterAddr, err := url.Parse(c.ClusterAddr())
if err != nil {
return err
}
c.postRecoveryUnsealFuncs = append(c.postRecoveryUnsealFuncs, func() error {
return raftStorage.StartRecoveryCluster(context.Background(), raft.Peer{
ID: raftStorage.NodeID(),
Address: parsedClusterAddr.Host,
})
})
return nil
}
// Initialized checks if the Vault is already initialized. This means one of
// two things: either the barrier has been created (with keyring and master key)
// and the seal config written to storage, or Raft is forming a cluster and a
// join/bootstrap is in progress.
func (c *Core) Initialized(ctx context.Context) (bool, error) {
// Check the barrier first
init, err := c.InitializedLocally(ctx)
if err != nil || init {
return init, err
}
if c.isRaftUnseal() {
return true, nil
}
rb := c.getRaftBackend()
if rb != nil && rb.Initialized() {
return true, nil
}
return false, nil
}
// InitializedLocally checks if the Vault is already initialized from the
// local node's perspective. This is the same thing as Initialized, unless
// using Raft, in which case Initialized may return true (because a peer
// we're joining to has been initialized) while InitializedLocally returns
// false (because we're not done bootstrapping raft on the local node).
func (c *Core) InitializedLocally(ctx context.Context) (bool, error) {
2016-04-04 17:44:22 +03:00
// Check the barrier first
init, err := c.barrier.Initialized(ctx)
2016-04-04 17:44:22 +03:00
if err != nil {
c.logger.Error("barrier init check failed", "error", err)
2016-04-04 17:44:22 +03:00
return false, err
}
if !init {
c.logger.Info("security barrier not initialized")
2016-04-04 17:44:22 +03:00
return false, nil
}
// Verify the seal configuration
sealConf, err := c.seal.BarrierConfig(ctx)
2016-04-04 17:44:22 +03:00
if err != nil {
return false, err
}
if sealConf == nil {
2016-08-19 23:45:17 +03:00
return false, fmt.Errorf("core: barrier reports initialized but no seal configuration found")
2016-04-04 17:44:22 +03:00
}
2016-04-04 17:44:22 +03:00
return true, nil
}
func (c *Core) generateShares(sc *SealConfig) ([]byte, [][]byte, error) {
// Generate a root key
rootKey, err := c.barrier.GenerateKey(c.secureRandomReader)
2016-04-04 17:44:22 +03:00
if err != nil {
return nil, nil, fmt.Errorf("key generation failed: %w", err)
2016-04-04 17:44:22 +03:00
}
// Return the root key if only a single key part is used
2016-04-04 17:44:22 +03:00
var unsealKeys [][]byte
if sc.SecretShares == 1 {
unsealKeys = append(unsealKeys, rootKey)
2016-04-04 17:44:22 +03:00
} else {
// Split the root key using the Shamir algorithm
shares, err := shamir.Split(rootKey, sc.SecretShares, sc.SecretThreshold)
2016-04-04 17:44:22 +03:00
if err != nil {
return nil, nil, fmt.Errorf("failed to generate barrier shares: %w", err)
2016-04-04 17:44:22 +03:00
}
unsealKeys = shares
}
// If we have PGP keys, perform the encryption
if len(sc.PGPKeys) > 0 {
hexEncodedShares := make([][]byte, len(unsealKeys))
for i := range unsealKeys {
2016-04-04 17:44:22 +03:00
hexEncodedShares[i] = []byte(hex.EncodeToString(unsealKeys[i]))
}
_, encryptedShares, err := pgpkeys.EncryptShares(hexEncodedShares, sc.PGPKeys)
if err != nil {
return nil, nil, err
}
unsealKeys = encryptedShares
}
return rootKey, unsealKeys, nil
2016-04-04 17:44:22 +03:00
}
// Initialize is used to initialize the Vault with the given
// configurations.
func (c *Core) Initialize(ctx context.Context, initParams *InitParams) (*InitResult, error) {
if err := LicenseInitCheck(c); err != nil {
return nil, err
}
2019-02-19 23:03:02 +03:00
atomic.StoreUint32(&initInProgress, 1)
defer atomic.StoreUint32(&initInProgress, 0)
barrierConfig := initParams.BarrierConfig
recoveryConfig := initParams.RecoveryConfig
// N.B. Although the core is capable of handling situations where some keys
// are stored and some aren't, in practice, replication + HSMs makes this
// extremely hard to reason about, to the point that it will probably never
// be supported. The reason is that each HSM needs to encode the root key
// separately, which means the shares must be generated independently,
// which means both that the shares will be different *AND* there would
// need to be a way to actually allow fetching of the generated keys by
// operators.
if c.SealAccess().StoredKeysSupported() == seal.StoredKeysSupportedGeneric {
if len(barrierConfig.PGPKeys) > 0 {
return nil, fmt.Errorf("PGP keys not supported when storing shares")
}
barrierConfig.SecretShares = 1
barrierConfig.SecretThreshold = 1
if barrierConfig.StoredShares != 1 {
c.Logger().Warn("stored keys supported on init, forcing shares/threshold to 1")
}
}
if initParams.LegacyShamirSeal {
barrierConfig.StoredShares = 0
} else {
barrierConfig.StoredShares = 1
}
if len(barrierConfig.PGPKeys) > 0 && len(barrierConfig.PGPKeys) != barrierConfig.SecretShares {
return nil, fmt.Errorf("incorrect number of PGP keys")
}
if c.SealAccess().RecoveryKeySupported() {
if len(recoveryConfig.PGPKeys) > 0 && len(recoveryConfig.PGPKeys) != recoveryConfig.SecretShares {
return nil, fmt.Errorf("incorrect number of PGP keys for recovery")
}
}
if c.seal.RecoveryKeySupported() {
2016-04-04 17:44:22 +03:00
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")
2016-04-04 17:44:22 +03:00
}
2016-05-15 19:58:36 +03:00
// Check if the seal configuration is valid
if err := recoveryConfig.Validate(); err != nil {
c.logger.Error("invalid recovery configuration", "error", err)
return nil, fmt.Errorf("invalid recovery configuration: %w", err)
2016-04-04 17:44:22 +03:00
}
}
2016-05-15 19:58:36 +03:00
// Check if the seal configuration is valid
2016-04-04 17:44:22 +03:00
if err := barrierConfig.Validate(); err != nil {
c.logger.Error("invalid seal configuration", "error", err)
return nil, fmt.Errorf("invalid seal configuration: %w", err)
2016-04-04 17:44:22 +03:00
}
// Avoid an initialization race
c.stateLock.Lock()
defer c.stateLock.Unlock()
// Check if we are initialized
init, err := c.Initialized(ctx)
2016-04-04 17:44:22 +03:00
if err != nil {
return nil, err
}
if init {
return nil, ErrAlreadyInit
}
raft: add support for using backend for ha_storage (#9193) * raft: initial work on raft ha storage support * add note on join * add todo note * raft: add support for bootstrapping and joining existing nodes * raft: gate bootstrap join by reading leader api address from storage * raft: properly check for raft-only for certain conditionals * raft: add bootstrap to api and cli * raft: fix bootstrap cli command * raft: add test for setting up new cluster with raft HA * raft: extend TestRaft_HA_NewCluster to include inmem and consul backends * raft: add test for updating an existing cluster to use raft HA * raft: remove debug log lines, clean up verifyRaftPeers * raft: minor cleanup * raft: minor cleanup * Update physical/raft/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/ha.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/ha.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/logical_system_raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * address feedback comments * address feedback comments * raft: refactor tls keyring logic * address feedback comments * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * address feedback comments * testing: fix import ordering * raft: rename var, cleanup comment line * docs: remove ha_storage restriction note on raft * docs: more raft HA interaction updates with migration and recovery mode * docs: update the raft join command * raft: update comments * raft: add missing isRaftHAOnly check for clearing out state set earlier * raft: update a few ha_storage config checks * Update command/operator_raft_bootstrap.go Co-authored-by: Vishal Nayak <vishalnayak@users.noreply.github.com> * raft: address feedback comments * raft: fix panic when checking for config.HAStorage.Type * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * Update website/pages/docs/commands/operator/raft.mdx Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * raft: remove bootstrap cli command * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * raft: address review feedback * raft: revert vendored sdk * raft: don't send applied index and node ID info if we're HA-only Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> Co-authored-by: Vishal Nayak <vishalnayak@users.noreply.github.com>
2020-06-23 22:04:13 +03:00
// Bootstrap the raft backend if that's provided as the physical or
// HA backend.
raftBackend := c.getRaftBackend()
if raftBackend != nil {
err := c.RaftBootstrap(ctx, true)
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 22:14:58 +03:00
if err != nil {
raft: add support for using backend for ha_storage (#9193) * raft: initial work on raft ha storage support * add note on join * add todo note * raft: add support for bootstrapping and joining existing nodes * raft: gate bootstrap join by reading leader api address from storage * raft: properly check for raft-only for certain conditionals * raft: add bootstrap to api and cli * raft: fix bootstrap cli command * raft: add test for setting up new cluster with raft HA * raft: extend TestRaft_HA_NewCluster to include inmem and consul backends * raft: add test for updating an existing cluster to use raft HA * raft: remove debug log lines, clean up verifyRaftPeers * raft: minor cleanup * raft: minor cleanup * Update physical/raft/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/ha.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/ha.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/logical_system_raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * address feedback comments * address feedback comments * raft: refactor tls keyring logic * address feedback comments * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * address feedback comments * testing: fix import ordering * raft: rename var, cleanup comment line * docs: remove ha_storage restriction note on raft * docs: more raft HA interaction updates with migration and recovery mode * docs: update the raft join command * raft: update comments * raft: add missing isRaftHAOnly check for clearing out state set earlier * raft: update a few ha_storage config checks * Update command/operator_raft_bootstrap.go Co-authored-by: Vishal Nayak <vishalnayak@users.noreply.github.com> * raft: address feedback comments * raft: fix panic when checking for config.HAStorage.Type * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * Update website/pages/docs/commands/operator/raft.mdx Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * raft: remove bootstrap cli command * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * raft: address review feedback * raft: revert vendored sdk * raft: don't send applied index and node ID info if we're HA-only Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> Co-authored-by: Vishal Nayak <vishalnayak@users.noreply.github.com>
2020-06-23 22:04:13 +03:00
c.logger.Error("failed to bootstrap raft", "error", err)
return nil, err
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 22:14:58 +03:00
}
raft: add support for using backend for ha_storage (#9193) * raft: initial work on raft ha storage support * add note on join * add todo note * raft: add support for bootstrapping and joining existing nodes * raft: gate bootstrap join by reading leader api address from storage * raft: properly check for raft-only for certain conditionals * raft: add bootstrap to api and cli * raft: fix bootstrap cli command * raft: add test for setting up new cluster with raft HA * raft: extend TestRaft_HA_NewCluster to include inmem and consul backends * raft: add test for updating an existing cluster to use raft HA * raft: remove debug log lines, clean up verifyRaftPeers * raft: minor cleanup * raft: minor cleanup * Update physical/raft/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/ha.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/ha.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/logical_system_raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * address feedback comments * address feedback comments * raft: refactor tls keyring logic * address feedback comments * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * address feedback comments * testing: fix import ordering * raft: rename var, cleanup comment line * docs: remove ha_storage restriction note on raft * docs: more raft HA interaction updates with migration and recovery mode * docs: update the raft join command * raft: update comments * raft: add missing isRaftHAOnly check for clearing out state set earlier * raft: update a few ha_storage config checks * Update command/operator_raft_bootstrap.go Co-authored-by: Vishal Nayak <vishalnayak@users.noreply.github.com> * raft: address feedback comments * raft: fix panic when checking for config.HAStorage.Type * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * Update website/pages/docs/commands/operator/raft.mdx Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * raft: remove bootstrap cli command * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * raft: address review feedback * raft: revert vendored sdk * raft: don't send applied index and node ID info if we're HA-only Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> Co-authored-by: Vishal Nayak <vishalnayak@users.noreply.github.com>
2020-06-23 22:04:13 +03:00
// Teardown cluster after bootstrap setup
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 22:14:58 +03:00
defer func() {
raft: add support for using backend for ha_storage (#9193) * raft: initial work on raft ha storage support * add note on join * add todo note * raft: add support for bootstrapping and joining existing nodes * raft: gate bootstrap join by reading leader api address from storage * raft: properly check for raft-only for certain conditionals * raft: add bootstrap to api and cli * raft: fix bootstrap cli command * raft: add test for setting up new cluster with raft HA * raft: extend TestRaft_HA_NewCluster to include inmem and consul backends * raft: add test for updating an existing cluster to use raft HA * raft: remove debug log lines, clean up verifyRaftPeers * raft: minor cleanup * raft: minor cleanup * Update physical/raft/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/ha.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/ha.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/logical_system_raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * address feedback comments * address feedback comments * raft: refactor tls keyring logic * address feedback comments * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * address feedback comments * testing: fix import ordering * raft: rename var, cleanup comment line * docs: remove ha_storage restriction note on raft * docs: more raft HA interaction updates with migration and recovery mode * docs: update the raft join command * raft: update comments * raft: add missing isRaftHAOnly check for clearing out state set earlier * raft: update a few ha_storage config checks * Update command/operator_raft_bootstrap.go Co-authored-by: Vishal Nayak <vishalnayak@users.noreply.github.com> * raft: address feedback comments * raft: fix panic when checking for config.HAStorage.Type * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * Update website/pages/docs/commands/operator/raft.mdx Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * raft: remove bootstrap cli command * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * raft: address review feedback * raft: revert vendored sdk * raft: don't send applied index and node ID info if we're HA-only Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> Co-authored-by: Vishal Nayak <vishalnayak@users.noreply.github.com>
2020-06-23 22:04:13 +03:00
if err := raftBackend.TeardownCluster(nil); err != nil {
c.logger.Error("failed to stop raft", "error", err)
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 22:14:58 +03:00
}
}()
}
err = c.seal.Init(ctx)
2016-04-04 17:44:22 +03:00
if err != nil {
c.logger.Error("failed to initialize seal", "error", err)
return nil, fmt.Errorf("error initializing seal: %w", err)
2016-04-04 17:44:22 +03:00
}
initPTCleanup := initPTFunc(c)
if initPTCleanup != nil {
defer initPTCleanup()
}
barrierKey, barrierKeyShares, err := c.generateShares(barrierConfig)
2016-04-04 17:44:22 +03:00
if err != nil {
c.logger.Error("error generating shares", "error", err)
2016-04-04 17:44:22 +03:00
return nil, err
}
var sealKey []byte
var sealKeyShares [][]byte
if barrierConfig.StoredShares == 1 && c.seal.BarrierType() == wrapping.WrapperTypeShamir {
sealKey, sealKeyShares, err = c.generateShares(barrierConfig)
if err != nil {
c.logger.Error("error generating shares", "error", err)
return nil, err
}
2018-09-18 06:03:00 +03:00
}
2016-04-04 17:44:22 +03:00
// Initialize the barrier
if err := c.barrier.Initialize(ctx, barrierKey, sealKey, c.secureRandomReader); err != nil {
c.logger.Error("failed to initialize barrier", "error", err)
return nil, fmt.Errorf("failed to initialize barrier: %w", err)
2016-04-04 17:44:22 +03:00
}
2016-08-19 23:45:17 +03:00
if c.logger.IsInfo() {
c.logger.Info("security barrier initialized", "stored", barrierConfig.StoredShares, "shares", barrierConfig.SecretShares, "threshold", barrierConfig.SecretThreshold)
2016-08-19 23:45:17 +03:00
}
2016-04-04 17:44:22 +03:00
// Unseal the barrier
if err := c.barrier.Unseal(ctx, barrierKey); err != nil {
c.logger.Error("failed to unseal barrier", "error", err)
return nil, fmt.Errorf("failed to unseal barrier: %w", err)
2016-04-04 17:44:22 +03:00
}
// 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.
2016-04-04 17:44:22 +03:00
if err := c.barrier.Seal(); err != nil {
c.logger.Error("failed to seal barrier", "error", err)
2016-04-04 17:44:22 +03:00
}
}()
err = c.seal.SetBarrierConfig(ctx, barrierConfig)
if err != nil {
c.logger.Error("failed to save barrier configuration", "error", err)
return nil, fmt.Errorf("barrier configuration saving failed: %w", err)
}
results := &InitResult{
SecretShares: [][]byte{},
}
// If we are storing shares, pop them out of the returned results and push
// them through the seal
switch c.seal.StoredKeysSupported() {
case seal.StoredKeysSupportedShamirRoot:
keysToStore := [][]byte{barrierKey}
shamirWrapper, err := c.seal.GetShamirWrapper()
if err != nil {
return nil, err
}
if err := shamirWrapper.SetAesGcmKeyBytes(sealKey); err != nil {
c.logger.Error("failed to set seal key", "error", err)
return nil, fmt.Errorf("failed to set seal key: %w", err)
}
if err := c.seal.SetStoredKeys(ctx, keysToStore); err != nil {
c.logger.Error("failed to store keys", "error", err)
return nil, fmt.Errorf("failed to store keys: %w", err)
}
results.SecretShares = sealKeyShares
case seal.StoredKeysSupportedGeneric:
keysToStore := [][]byte{barrierKey}
if err := c.seal.SetStoredKeys(ctx, keysToStore); err != nil {
c.logger.Error("failed to store keys", "error", err)
return nil, fmt.Errorf("failed to store keys: %w", err)
}
default:
// We don't support initializing an old-style Shamir seal anymore, so
// this case is only reachable by tests.
results.SecretShares = barrierKeyShares
}
2016-04-04 17:44:22 +03:00
// Perform initial setup
if err := c.setupCluster(ctx); err != nil {
c.logger.Error("cluster setup failed during init", "error", err)
return nil, err
}
2018-09-18 06:03:00 +03:00
// 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)
2016-04-04 17:44:22 +03:00
return nil, err
}
2016-04-15 20:04:29 +03:00
// 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)
2016-04-15 20:04:29 +03:00
if err != nil {
c.logger.Error("failed to save recovery configuration", "error", err)
return nil, fmt.Errorf("recovery configuration saving failed: %w", err)
2016-04-15 20:04:29 +03:00
}
if recoveryConfig.SecretShares > 0 {
recoveryKey, recoveryUnsealKeys, err := c.generateShares(recoveryConfig)
if err != nil {
c.logger.Error("failed to generate recovery shares", "error", err)
2016-04-15 20:04:29 +03:00
return nil, err
}
err = c.seal.SetRecoveryKey(ctx, recoveryKey)
2016-04-15 20:04:29 +03:00
if err != nil {
return nil, err
}
results.RecoveryShares = recoveryUnsealKeys
}
}
2016-04-04 17:44:22 +03:00
// Generate a new root token
rootToken, err := c.tokenStore.rootToken(ctx)
2016-04-04 17:44:22 +03:00
if err != nil {
c.logger.Error("root token generation failed", "error", err)
2016-04-04 17:44:22 +03:00
return nil, err
}
results.RootToken = rootToken.ExternalID
c.logger.Info("root token generated")
2016-04-04 17:44:22 +03:00
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])
}
raft: add support for using backend for ha_storage (#9193) * raft: initial work on raft ha storage support * add note on join * add todo note * raft: add support for bootstrapping and joining existing nodes * raft: gate bootstrap join by reading leader api address from storage * raft: properly check for raft-only for certain conditionals * raft: add bootstrap to api and cli * raft: fix bootstrap cli command * raft: add test for setting up new cluster with raft HA * raft: extend TestRaft_HA_NewCluster to include inmem and consul backends * raft: add test for updating an existing cluster to use raft HA * raft: remove debug log lines, clean up verifyRaftPeers * raft: minor cleanup * raft: minor cleanup * Update physical/raft/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/ha.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/ha.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/logical_system_raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * address feedback comments * address feedback comments * raft: refactor tls keyring logic * address feedback comments * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * address feedback comments * testing: fix import ordering * raft: rename var, cleanup comment line * docs: remove ha_storage restriction note on raft * docs: more raft HA interaction updates with migration and recovery mode * docs: update the raft join command * raft: update comments * raft: add missing isRaftHAOnly check for clearing out state set earlier * raft: update a few ha_storage config checks * Update command/operator_raft_bootstrap.go Co-authored-by: Vishal Nayak <vishalnayak@users.noreply.github.com> * raft: address feedback comments * raft: fix panic when checking for config.HAStorage.Type * Update vault/raft.go Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * Update website/pages/docs/commands/operator/raft.mdx Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> * raft: remove bootstrap cli command * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * Update vault/raft.go Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> * raft: address review feedback * raft: revert vendored sdk * raft: don't send applied index and node ID info if we're HA-only Co-authored-by: Brian Kassouf <briankassouf@users.noreply.github.com> Co-authored-by: Alexander Bezobchuk <alexanderbez@users.noreply.github.com> Co-authored-by: Vishal Nayak <vishalnayak@users.noreply.github.com>
2020-06-23 22:04:13 +03:00
if raftBackend != nil {
if _, err := c.raftCreateTLSKeyring(ctx); err != nil {
c.logger.Error("failed to create raft TLS keyring", "error", err)
return nil, err
}
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 22:14:58 +03:00
}
2016-04-04 17:44:22 +03:00
// Prepare to re-seal
if err := c.preSeal(); err != nil {
c.logger.Error("pre-seal teardown failed", "error", err)
2016-04-04 17:44:22 +03:00
return nil, err
}
if c.serviceRegistration != nil {
if err := c.serviceRegistration.NotifyInitializedStateChange(true); err != nil {
if c.logger.IsWarn() {
c.logger.Warn("notification of initialization failed", "error", err)
}
}
}
2016-04-04 17:44:22 +03:00
return results, nil
}
// UnsealWithStoredKeys performs auto-unseal using stored keys. An error
// return value of "nil" implies the Vault instance is unsealed.
//
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 22:14:58 +03:00
// 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.BarrierType() == wrapping.WrapperTypeShamir {
2016-04-04 17:44:22 +03:00
return nil
}
2018-10-23 09:34:02 +03:00
// Disallow auto-unsealing when migrating
if c.IsInSealMigrationMode(true) && !c.IsSealMigrated(true) {
return NewNonFatalError(errors.New("cannot auto-unseal during seal migration"))
2018-10-23 09:34:02 +03:00
}
c.stateLock.Lock()
defer c.stateLock.Unlock()
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)
2016-04-04 17:44:22 +03:00
if err != nil {
return NewNonFatalError(fmt.Errorf("fetching stored unseal keys failed: %w", err))
2016-04-04 17:44:22 +03:00
}
// This usually happens when auto-unseal is configured, but the servers have
// not been initialized yet.
2016-04-04 17:44:22 +03:00
if len(keys) == 0 {
return NewNonFatalError(errors.New("stored unseal keys are supported, but none were found"))
}
if len(keys) != 1 {
return NewNonFatalError(errors.New("expected exactly one stored key"))
}
err = c.unsealInternal(ctx, keys[0])
if err != nil {
return NewNonFatalError(fmt.Errorf("unseal with stored key failed: %w", err))
2016-04-04 17:44:22 +03:00
}
2017-10-23 23:49:46 +03:00
if c.Sealed() {
// 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 key")
} else {
c.Logger().Info("unsealed with stored key")
}
2016-04-04 17:44:22 +03:00
return nil
}