2019-06-20 19:14:58 +00:00
package vault
import (
"context"
"encoding/base64"
"errors"
"fmt"
"net/http"
"net/url"
"strings"
"sync"
2019-06-27 17:00:03 +00:00
"sync/atomic"
2019-06-20 19:14:58 +00:00
"time"
2020-01-14 01:02:16 +00:00
"github.com/golang/protobuf/proto"
2019-06-20 19:14:58 +00:00
cleanhttp "github.com/hashicorp/go-cleanhttp"
2020-10-13 20:26:39 +00:00
"github.com/hashicorp/go-discover"
2020-10-23 20:13:09 +00:00
discoverk8s "github.com/hashicorp/go-discover/provider/k8s"
2020-06-23 19:04:13 +00:00
"github.com/hashicorp/go-hclog"
2020-01-11 01:39:52 +00:00
wrapping "github.com/hashicorp/go-kms-wrapping"
2021-07-16 00:17:31 +00:00
"github.com/hashicorp/go-secure-stdlib/tlsutil"
2019-06-20 19:14:58 +00:00
uuid "github.com/hashicorp/go-uuid"
"github.com/hashicorp/vault/api"
"github.com/hashicorp/vault/physical/raft"
"github.com/hashicorp/vault/sdk/helper/jsonutil"
"github.com/hashicorp/vault/sdk/logical"
"github.com/hashicorp/vault/vault/seal"
"github.com/mitchellh/mapstructure"
"golang.org/x/net/http2"
)
var (
raftTLSStoragePath = "core/raft/tls"
raftTLSRotationPeriod = 24 * time . Hour
2020-06-23 19:04:13 +00:00
2021-03-03 18:59:50 +00:00
raftAutopilotConfigurationStoragePath = "core/raft/autopilot/configuration"
2020-06-23 19:04:13 +00:00
// TestingUpdateClusterAddr is used in tests to override the cluster address
TestingUpdateClusterAddr uint32
2021-05-19 20:07:58 +00:00
ErrJoinWithoutAutoloading = errors . New ( "attempt to join a cluster using autoloaded licenses while not using autoloading ourself" )
2019-06-20 19:14:58 +00:00
)
2021-04-20 22:25:04 +00:00
// GetRaftNodeID returns the raft node ID if there is one, or an empty string if there's not
func ( c * Core ) GetRaftNodeID ( ) string {
rb := c . getRaftBackend ( )
if rb == nil {
return ""
} else {
return rb . NodeID ( )
}
}
2020-05-18 23:07:27 +00:00
func ( c * Core ) GetRaftIndexes ( ) ( committed uint64 , applied uint64 ) {
c . stateLock . RLock ( )
defer c . stateLock . RUnlock ( )
raftStorage , ok := c . underlyingPhysical . ( * raft . RaftBackend )
if ! ok {
return 0 , 0
}
return raftStorage . CommittedIndex ( ) , raftStorage . AppliedIndex ( )
}
2020-06-23 19:04:13 +00:00
// startRaftBackend will call SetupCluster in the raft backend which starts raft
2019-06-20 19:14:58 +00:00
// up and enables the cluster handler.
2020-06-23 19:04:13 +00:00
func ( c * Core ) startRaftBackend ( ctx context . Context ) ( retErr error ) {
raftBackend := c . getRaftBackend ( )
2021-03-03 18:59:50 +00:00
if raftBackend == nil {
2019-06-20 19:14:58 +00:00
return nil
}
// Retrieve the raft TLS information
raftTLSEntry , err := c . barrier . Get ( ctx , raftTLSStoragePath )
if err != nil {
return err
}
2019-07-29 20:05:43 +00:00
var creating bool
var raftTLS * raft . TLSKeyring
switch raftTLSEntry {
case nil :
2020-06-23 19:04:13 +00:00
// If this is HA-only and no TLS keyring is found, that means the
// cluster has not been bootstrapped or joined. We return early here in
// this case. If we return here, the raft object has not been instantiated,
// and a bootstrap call should be made.
if c . isRaftHAOnly ( ) {
c . logger . Trace ( "skipping raft backend setup during unseal, no bootstrap operation has been started yet" )
return nil
}
2019-07-29 20:05:43 +00:00
// If we did not find a TLS keyring we will attempt to create one here.
// This happens after a storage migration process. This node is also
// marked to start as leader so we can write the new TLS Key. This is an
// error condition if there are already multiple nodes in the cluster,
// and the below storage write will fail. If the cluster is somehow in
// this state the unseal will fail and a cluster recovery will need to
// be done.
creating = true
2019-10-17 17:33:00 +00:00
raftTLSKey , err := raft . GenerateTLSKey ( c . secureRandomReader )
2019-07-29 20:05:43 +00:00
if err != nil {
return err
}
raftTLS = & raft . TLSKeyring {
Keys : [ ] * raft . TLSKey { raftTLSKey } ,
ActiveKeyID : raftTLSKey . ID ,
}
default :
raftTLS = new ( raft . TLSKeyring )
if err := raftTLSEntry . DecodeJSON ( raftTLS ) ; err != nil {
return err
}
2019-06-20 19:14:58 +00:00
}
2020-06-23 19:04:13 +00:00
hasState , err := raftBackend . HasState ( )
if err != nil {
return err
}
// This can be hit on follower nodes that got their config updated to use
// raft for HA-only before they are joined to the cluster. Since followers
// in this case use shared storage, it doesn't return early from the TLS
// case above, but there's not raft state yet for the backend to call
// raft.SetupCluster.
if ! hasState {
c . logger . Trace ( "skipping raft backend setup during unseal, no raft state found" )
return nil
}
raftBackend . SetRestoreCallback ( c . raftSnapshotRestoreCallback ( true , true ) )
2021-03-03 18:59:50 +00:00
2020-06-23 19:04:13 +00:00
if err := raftBackend . SetupCluster ( ctx , raft . SetupOpts {
2019-07-29 20:05:43 +00:00
TLSKeyring : raftTLS ,
2019-09-03 15:59:56 +00:00
ClusterListener : c . getClusterListener ( ) ,
2019-07-29 20:05:43 +00:00
StartAsLeader : creating ,
} ) ; err != nil {
2019-06-20 19:14:58 +00:00
return err
}
2019-07-29 20:05:43 +00:00
defer func ( ) {
if retErr != nil {
c . logger . Info ( "stopping raft server" )
2020-06-23 19:04:13 +00:00
if err := raftBackend . TeardownCluster ( c . getClusterListener ( ) ) ; err != nil {
2019-07-29 20:05:43 +00:00
c . logger . Error ( "failed to stop raft server" , "error" , err )
}
}
} ( )
// If we are in need of creating the TLS keyring then we should write it out
// to storage here. If we fail it may mean we couldn't become leader and we
// should error out.
if creating {
c . logger . Info ( "writing raft TLS keyring to storage" )
entry , err := logical . StorageEntryJSON ( raftTLSStoragePath , raftTLS )
if err != nil {
c . logger . Error ( "error marshaling raft TLS keyring" , "error" , err )
return err
}
if err := c . barrier . Put ( ctx , entry ) ; err != nil {
c . logger . Error ( "error writing raft TLS keyring" , "error" , err )
return err
}
}
2019-06-20 19:14:58 +00:00
return nil
}
2019-07-03 20:56:30 +00:00
func ( c * Core ) setupRaftActiveNode ( ctx context . Context ) error {
2021-03-03 18:59:50 +00:00
raftBackend := c . getRaftBackend ( )
if raftBackend == nil {
return nil
}
c . logger . Info ( "starting raft active node" )
autopilotConfig , err := c . loadAutopilotConfiguration ( ctx )
if err != nil {
c . logger . Error ( "failed to load autopilot config from storage when setting up cluster; continuing since autopilot falls back to default config" , "error" , err )
}
2021-03-23 21:13:44 +00:00
disableAutopilot := c . disableAutopilot
raftBackend . SetupAutopilot ( c . activeContext , autopilotConfig , c . raftFollowerStates , disableAutopilot )
2021-03-03 18:59:50 +00:00
2020-05-19 01:22:25 +00:00
c . pendingRaftPeers = & sync . Map { }
2021-05-19 17:03:32 +00:00
// Reload the raft TLS keys to ensure we are using the latest version.
if err := c . checkRaftTLSKeyUpgrades ( ctx ) ; err != nil {
return err
}
2019-07-03 20:56:30 +00:00
return c . startPeriodicRaftTLSRotate ( ctx )
}
func ( c * Core ) stopRaftActiveNode ( ) {
2021-03-03 18:59:50 +00:00
raftBackend := c . getRaftBackend ( )
if raftBackend == nil {
return
}
c . logger . Info ( "stopping raft active node" )
if ! raftBackend . AutopilotDisabled ( ) {
raftBackend . StopAutopilot ( )
}
2019-07-03 20:56:30 +00:00
c . pendingRaftPeers = nil
c . stopPeriodicRaftTLSRotate ( )
}
2020-06-23 19:04:13 +00:00
func ( c * Core ) startPeriodicRaftTLSRotate ( ctx context . Context ) error {
raftBackend := c . getRaftBackend ( )
// No-op if raft is not being used
if raftBackend == nil {
return nil
}
c . raftTLSRotationStopCh = make ( chan struct { } )
logger := c . logger . Named ( "raft" )
if c . isRaftHAOnly ( ) {
return c . raftTLSRotateDirect ( ctx , logger , c . raftTLSRotationStopCh )
}
return c . raftTLSRotatePhased ( ctx , logger , raftBackend , c . raftTLSRotationStopCh )
}
// raftTLSRotateDirect will spawn a go routine in charge of periodically
// rotating the TLS certs and keys used for raft traffic.
//
// The logic for updating the TLS keyring is through direct storage update. This
// is called whenever raft is used for HA-only, which means that the underlying
// storage is a shared physical object, thus requiring no additional
// coordination.
func ( c * Core ) raftTLSRotateDirect ( ctx context . Context , logger hclog . Logger , stopCh chan struct { } ) error {
logger . Info ( "creating new raft TLS config" )
rotateKeyring := func ( ) ( time . Time , error ) {
// Create a new key
raftTLSKey , err := raft . GenerateTLSKey ( c . secureRandomReader )
if err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to generate new raft TLS key: %w" , err )
2020-06-23 19:04:13 +00:00
}
// Read the existing keyring
keyring , err := c . raftReadTLSKeyring ( ctx )
if err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to read raft TLS keyring: %w" , err )
2020-06-23 19:04:13 +00:00
}
// Advance the term and store the new key, replacing the old one.
// Unlike phased rotation, we don't need to update AppliedIndex since
// we don't rely on it to check whether the followers got the key. A
// shared storage means that followers will have the key as soon as it's
// written to storage.
keyring . Term += 1
keyring . Keys [ 0 ] = raftTLSKey
keyring . ActiveKeyID = raftTLSKey . ID
entry , err := logical . StorageEntryJSON ( raftTLSStoragePath , keyring )
if err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to json encode keyring: %w" , err )
2020-06-23 19:04:13 +00:00
}
if err := c . barrier . Put ( ctx , entry ) ; err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to write keyring: %w" , err )
2020-06-23 19:04:13 +00:00
}
logger . Info ( "wrote new raft TLS config" )
// Schedule the next rotation
return raftTLSKey . CreatedTime . Add ( raftTLSRotationPeriod ) , nil
}
// Read the keyring to calculate the time of next rotation.
keyring , err := c . raftReadTLSKeyring ( ctx )
if err != nil {
return err
}
activeKey := keyring . GetActive ( )
if activeKey == nil {
return errors . New ( "no active raft TLS key found" )
}
go func ( ) {
nextRotationTime := activeKey . CreatedTime . Add ( raftTLSRotationPeriod )
var backoff bool
for {
// If we encountered and error we should try to create the key
// again.
if backoff {
nextRotationTime = time . Now ( ) . Add ( 10 * time . Second )
backoff = false
}
select {
case <- time . After ( time . Until ( nextRotationTime ) ) :
// It's time to rotate the keys
next , err := rotateKeyring ( )
if err != nil {
logger . Error ( "failed to rotate TLS key" , "error" , err )
backoff = true
continue
}
nextRotationTime = next
case <- stopCh :
return
}
}
} ( )
return nil
}
// raftTLSRotatePhased will spawn a go routine in charge of periodically
2019-06-20 19:14:58 +00:00
// rotating the TLS certs and keys used for raft traffic.
//
// The logic for updating the TLS certificate uses a pseudo two phase commit
// using the known applied indexes from standby nodes. When writing a new Key
// it will be appended to the end of the keyring. Standbys can start accepting
// connections with this key as soon as they see the update. Then it will write
// the keyring a second time indicating the applied index for this key update.
//
// The active node will wait until it sees all standby nodes are at or past the
// applied index for this update. At that point it will delete the older key
// and make the new key active. The key isn't officially in use until this
// happens. The dual write ensures the standby at least gets the first update
// containing the key before the active node switches over to using it.
//
// If a standby is shut down then it cannot advance the key term until it
// receives the update. This ensures a standby node isn't left behind and unable
// to reconnect with the cluster. Additionally, only one outstanding key
// is allowed for this same reason (max keyring size of 2).
2020-06-23 19:04:13 +00:00
func ( c * Core ) raftTLSRotatePhased ( ctx context . Context , logger hclog . Logger , raftBackend * raft . RaftBackend , stopCh chan struct { } ) error {
2021-03-03 18:59:50 +00:00
followerStates := c . raftFollowerStates
followerStates . Clear ( )
2019-06-20 19:14:58 +00:00
// Pre-populate the follower list with the set of peers.
2020-06-23 19:04:13 +00:00
raftConfig , err := raftBackend . GetConfiguration ( ctx )
2019-06-20 19:14:58 +00:00
if err != nil {
return err
}
for _ , server := range raftConfig . Servers {
2020-06-23 19:04:13 +00:00
if server . NodeID != raftBackend . NodeID ( ) {
2021-03-03 18:59:50 +00:00
followerStates . Update ( server . NodeID , 0 , 0 , "voter" )
2019-06-20 19:14:58 +00:00
}
}
// rotateKeyring writes new key data to the keyring and adds an applied
// index that is used to verify it has been committed. The keys written in
// this function can be used on standbys but the active node doesn't start
// using it yet.
rotateKeyring := func ( ) ( time . Time , error ) {
// Read the existing keyring
2020-06-23 19:04:13 +00:00
keyring , err := c . raftReadTLSKeyring ( ctx )
2019-06-20 19:14:58 +00:00
if err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to read raft TLS keyring: %w" , err )
2019-06-20 19:14:58 +00:00
}
switch {
case len ( keyring . Keys ) == 2 && keyring . Keys [ 1 ] . AppliedIndex == 0 :
// If this case is hit then the second write to add the applied
// index failed. Attempt to write it again.
2020-06-23 19:04:13 +00:00
keyring . Keys [ 1 ] . AppliedIndex = raftBackend . AppliedIndex ( )
keyring . AppliedIndex = raftBackend . AppliedIndex ( )
2019-06-20 19:14:58 +00:00
entry , err := logical . StorageEntryJSON ( raftTLSStoragePath , keyring )
if err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to json encode keyring: %w" , err )
2019-06-20 19:14:58 +00:00
}
if err := c . barrier . Put ( ctx , entry ) ; err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to write keyring: %w" , err )
2019-06-20 19:14:58 +00:00
}
case len ( keyring . Keys ) > 1 :
// If there already exists a pending key update then the update
// hasn't replicated down to all standby nodes yet. Don't allow any
// new keys to be created until all standbys have seen this previous
2020-06-23 19:04:13 +00:00
// rotation. As a backoff strategy, another rotation attempt is
2019-06-20 19:14:58 +00:00
// scheduled for 5 minutes from now.
logger . Warn ( "skipping new raft TLS config creation, keys are pending" )
return time . Now ( ) . Add ( time . Minute * 5 ) , nil
}
logger . Info ( "creating new raft TLS config" )
// Create a new key
2019-10-17 17:33:00 +00:00
raftTLSKey , err := raft . GenerateTLSKey ( c . secureRandomReader )
2019-06-20 19:14:58 +00:00
if err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to generate new raft TLS key: %w" , err )
2019-06-20 19:14:58 +00:00
}
// Advance the term and store the new key
keyring . Term += 1
keyring . Keys = append ( keyring . Keys , raftTLSKey )
entry , err := logical . StorageEntryJSON ( raftTLSStoragePath , keyring )
if err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to json encode keyring: %w" , err )
2019-06-20 19:14:58 +00:00
}
if err := c . barrier . Put ( ctx , entry ) ; err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to write keyring: %w" , err )
2019-06-20 19:14:58 +00:00
}
// Write the keyring again with the new applied index. This allows us to
2020-06-23 19:04:13 +00:00
// track if standby nodes received the update.
keyring . Keys [ 1 ] . AppliedIndex = raftBackend . AppliedIndex ( )
keyring . AppliedIndex = raftBackend . AppliedIndex ( )
2019-06-20 19:14:58 +00:00
entry , err = logical . StorageEntryJSON ( raftTLSStoragePath , keyring )
if err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to json encode keyring: %w" , err )
2019-06-20 19:14:58 +00:00
}
if err := c . barrier . Put ( ctx , entry ) ; err != nil {
2021-05-11 17:12:54 +00:00
return time . Time { } , fmt . Errorf ( "failed to write keyring: %w" , err )
2019-06-20 19:14:58 +00:00
}
logger . Info ( "wrote new raft TLS config" )
// Schedule the next rotation
return raftTLSKey . CreatedTime . Add ( raftTLSRotationPeriod ) , nil
}
// checkCommitted verifies key updates have been applied to all nodes and
// finalizes the rotation by deleting the old keys and updating the raft
// backend.
checkCommitted := func ( ) error {
2020-06-23 19:04:13 +00:00
keyring , err := c . raftReadTLSKeyring ( ctx )
2019-06-20 19:14:58 +00:00
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to read raft TLS keyring: %w" , err )
2019-06-20 19:14:58 +00:00
}
switch {
case len ( keyring . Keys ) == 1 :
// No Keys to apply
return nil
case keyring . Keys [ 1 ] . AppliedIndex != keyring . AppliedIndex :
// We haven't fully committed the new key, continue here
return nil
2021-03-03 18:59:50 +00:00
case followerStates . MinIndex ( ) < keyring . AppliedIndex :
2019-06-20 19:14:58 +00:00
// Not all the followers have applied the latest key
return nil
}
// Upgrade to the new key
keyring . Keys = keyring . Keys [ 1 : ]
keyring . ActiveKeyID = keyring . Keys [ 0 ] . ID
keyring . Term += 1
entry , err := logical . StorageEntryJSON ( raftTLSStoragePath , keyring )
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to json encode keyring: %w" , err )
2019-06-20 19:14:58 +00:00
}
if err := c . barrier . Put ( ctx , entry ) ; err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to write keyring: %w" , err )
2019-06-20 19:14:58 +00:00
}
// Update the TLS Key in the backend
2020-06-23 19:04:13 +00:00
if err := raftBackend . SetTLSKeyring ( keyring ) ; err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to install keyring: %w" , err )
2019-06-20 19:14:58 +00:00
}
logger . Info ( "installed new raft TLS key" , "term" , keyring . Term )
return nil
}
// Read the keyring to calculate the time of next rotation.
2020-06-23 19:04:13 +00:00
keyring , err := c . raftReadTLSKeyring ( ctx )
2019-06-20 19:14:58 +00:00
if err != nil {
return err
}
activeKey := keyring . GetActive ( )
if activeKey == nil {
return errors . New ( "no active raft TLS key found" )
}
// Start the process in a go routine
go func ( ) {
nextRotationTime := activeKey . CreatedTime . Add ( raftTLSRotationPeriod )
keyCheckInterval := time . NewTicker ( 1 * time . Minute )
defer keyCheckInterval . Stop ( )
var backoff bool
for {
// If we encountered and error we should try to create the key
// again.
if backoff {
nextRotationTime = time . Now ( ) . Add ( 10 * time . Second )
backoff = false
}
select {
case <- keyCheckInterval . C :
err := checkCommitted ( )
if err != nil {
logger . Error ( "failed to activate TLS key" , "error" , err )
}
case <- time . After ( time . Until ( nextRotationTime ) ) :
// It's time to rotate the keys
next , err := rotateKeyring ( )
if err != nil {
logger . Error ( "failed to rotate TLS key" , "error" , err )
backoff = true
continue
}
nextRotationTime = next
case <- stopCh :
return
}
}
} ( )
return nil
}
2020-06-23 19:04:13 +00:00
func ( c * Core ) raftReadTLSKeyring ( ctx context . Context ) ( * raft . TLSKeyring , error ) {
tlsKeyringEntry , err := c . barrier . Get ( ctx , raftTLSStoragePath )
if err != nil {
return nil , err
}
if tlsKeyringEntry == nil {
return nil , errors . New ( "no keyring found" )
}
var keyring raft . TLSKeyring
if err := tlsKeyringEntry . DecodeJSON ( & keyring ) ; err != nil {
return nil , err
}
return & keyring , nil
}
// raftCreateTLSKeyring creates the initial TLS key and the TLS Keyring for raft
// use. If a keyring entry is already present in storage, it will return an
// error.
func ( c * Core ) raftCreateTLSKeyring ( ctx context . Context ) ( * raft . TLSKeyring , error ) {
if raftBackend := c . getRaftBackend ( ) ; raftBackend == nil {
return nil , fmt . Errorf ( "raft backend not in use" )
}
// Check if the keyring is already present
raftTLSEntry , err := c . barrier . Get ( ctx , raftTLSStoragePath )
if err != nil {
return nil , err
}
if raftTLSEntry != nil {
return nil , fmt . Errorf ( "TLS keyring already present" )
2019-07-29 20:05:43 +00:00
}
2019-10-17 17:33:00 +00:00
raftTLS , err := raft . GenerateTLSKey ( c . secureRandomReader )
2019-07-29 20:05:43 +00:00
if err != nil {
2020-06-23 19:04:13 +00:00
return nil , err
2019-07-29 20:05:43 +00:00
}
keyring := & raft . TLSKeyring {
Keys : [ ] * raft . TLSKey { raftTLS } ,
ActiveKeyID : raftTLS . ID ,
}
entry , err := logical . StorageEntryJSON ( raftTLSStoragePath , keyring )
if err != nil {
2020-06-23 19:04:13 +00:00
return nil , err
2019-07-29 20:05:43 +00:00
}
if err := c . barrier . Put ( ctx , entry ) ; err != nil {
2020-06-23 19:04:13 +00:00
return nil , err
2019-07-29 20:05:43 +00:00
}
2020-06-23 19:04:13 +00:00
return keyring , nil
2019-07-29 20:05:43 +00:00
}
2019-06-20 19:14:58 +00:00
func ( c * Core ) stopPeriodicRaftTLSRotate ( ) {
if c . raftTLSRotationStopCh != nil {
close ( c . raftTLSRotationStopCh )
}
c . raftTLSRotationStopCh = nil
2021-03-03 18:59:50 +00:00
c . raftFollowerStates . Clear ( )
2019-06-20 19:14:58 +00:00
}
func ( c * Core ) checkRaftTLSKeyUpgrades ( ctx context . Context ) error {
2020-06-23 19:04:13 +00:00
raftBackend := c . getRaftBackend ( )
if raftBackend == nil {
2019-06-20 19:14:58 +00:00
return nil
}
tlsKeyringEntry , err := c . barrier . Get ( ctx , raftTLSStoragePath )
if err != nil {
return err
}
if tlsKeyringEntry == nil {
return nil
}
2019-07-29 20:05:43 +00:00
var keyring raft . TLSKeyring
2019-06-20 19:14:58 +00:00
if err := tlsKeyringEntry . DecodeJSON ( & keyring ) ; err != nil {
return err
}
2020-06-23 19:04:13 +00:00
if err := raftBackend . SetTLSKeyring ( & keyring ) ; err != nil {
2019-06-20 19:14:58 +00:00
return err
}
return nil
}
// handleSnapshotRestore is for the raft backend to hook back into core after a
// snapshot is restored so we can clear the necessary caches and handle changing
// keyrings or master keys
2019-07-03 20:56:30 +00:00
func ( c * Core ) raftSnapshotRestoreCallback ( grabLock bool , sealNode bool ) func ( context . Context ) error {
return func ( ctx context . Context ) ( retErr error ) {
2019-06-20 19:14:58 +00:00
c . logger . Info ( "running post snapshot restore invalidations" )
if grabLock {
// Grab statelock
2020-07-21 12:34:07 +00:00
if stopped := grabLockOrStop ( c . stateLock . Lock , c . stateLock . Unlock , c . standbyStopCh . Load ( ) . ( chan struct { } ) ) ; stopped {
2019-06-20 19:14:58 +00:00
c . logger . Error ( "did not apply snapshot; vault is shutting down" )
return errors . New ( "did not apply snapshot; vault is shutting down" )
}
defer c . stateLock . Unlock ( )
}
2019-07-03 20:56:30 +00:00
if sealNode {
// If we failed to restore the snapshot we should seal this node as
// it's in an unknown state
defer func ( ) {
if retErr != nil {
if err := c . sealInternalWithOptions ( false , false , true ) ; err != nil {
c . logger . Error ( "failed to seal node" , "error" , err )
}
}
} ( )
}
2019-06-20 19:14:58 +00:00
// Purge the cache so we make sure we are operating on fresh data
c . physicalCache . Purge ( ctx )
// Reload the keyring in case it changed. If this fails it's likely
// we've changed master keys.
err := c . performKeyUpgrades ( ctx )
if err != nil {
// The snapshot contained a master key or keyring we couldn't
// recover
switch c . seal . BarrierType ( ) {
2020-01-11 01:39:52 +00:00
case wrapping . Shamir :
2019-06-20 19:14:58 +00:00
// If we are a shamir seal we can't do anything. Just
// seal all nodes.
// Seal ourselves
c . logger . Info ( "failed to perform key upgrades, sealing" , "error" , err )
return err
2020-01-11 01:39:52 +00:00
2019-06-20 19:14:58 +00:00
default :
// If we are using an auto-unseal we can try to use the seal to
// unseal again. If the auto-unseal mechanism has changed then
// there isn't anything we can do but seal.
c . logger . Info ( "failed to perform key upgrades, reloading using auto seal" )
keys , err := c . seal . GetStoredKeys ( ctx )
if err != nil {
c . logger . Error ( "raft snapshot restore failed to get stored keys" , "error" , err )
return err
}
if err := c . barrier . Seal ( ) ; err != nil {
c . logger . Error ( "raft snapshot restore failed to seal barrier" , "error" , err )
return err
}
if err := c . barrier . Unseal ( ctx , keys [ 0 ] ) ; err != nil {
c . logger . Error ( "raft snapshot restore failed to unseal barrier" , "error" , err )
return err
}
c . logger . Info ( "done reloading master key using auto seal" )
}
}
2020-07-21 17:59:07 +00:00
// Refresh the raft TLS keys
if err := c . checkRaftTLSKeyUpgrades ( ctx ) ; err != nil {
c . logger . Info ( "failed to perform TLS key upgrades, sealing" , "error" , err )
return err
}
2019-06-20 19:14:58 +00:00
return nil
}
}
2020-01-14 01:02:16 +00:00
func ( c * Core ) InitiateRetryJoin ( ctx context . Context ) error {
2020-06-23 19:04:13 +00:00
raftBackend := c . getRaftBackend ( )
if raftBackend == nil {
return nil
2020-01-14 01:02:16 +00:00
}
2020-06-23 19:04:13 +00:00
if raftBackend . Initialized ( ) {
2020-01-14 01:02:16 +00:00
return nil
}
2020-06-23 19:04:13 +00:00
leaderInfos , err := raftBackend . JoinConfig ( )
2020-01-14 01:02:16 +00:00
if err != nil {
return err
2019-06-20 19:14:58 +00:00
}
2020-01-14 01:02:16 +00:00
// Nothing to do if config wasn't supplied
if len ( leaderInfos ) == 0 {
return nil
}
c . logger . Info ( "raft retry join initiated" )
if _ , err = c . JoinRaftCluster ( ctx , leaderInfos , false ) ; err != nil {
return err
}
return nil
}
2021-02-10 21:41:58 +00:00
func ( c * Core ) JoinRaftCluster ( ctx context . Context , leaderInfos [ ] * raft . LeaderJoinInfo , nonVoter bool ) ( bool , error ) {
2020-06-23 19:04:13 +00:00
raftBackend := c . getRaftBackend ( )
if raftBackend == nil {
return false , errors . New ( "raft backend not in use" )
2019-06-20 19:14:58 +00:00
}
2021-03-03 18:59:50 +00:00
if err := raftBackend . SetDesiredSuffrage ( nonVoter ) ; err != nil {
c . logger . Error ( "failed to set desired suffrage for this node" , "error" , err )
return false , nil
}
2020-12-08 18:55:34 +00:00
init , err := c . InitializedLocally ( ctx )
2019-06-20 19:14:58 +00:00
if err != nil {
2021-05-11 17:12:54 +00:00
return false , fmt . Errorf ( "failed to check if core is initialized: %w" , err )
2019-06-20 19:14:58 +00:00
}
2020-06-23 19:04:13 +00:00
isRaftHAOnly := c . isRaftHAOnly ( )
// Prevent join from happening if we're using raft for storage and
// it has already been initialized.
if init && ! isRaftHAOnly {
2020-01-14 01:02:16 +00:00
return true , nil
2019-06-20 19:14:58 +00:00
}
2020-06-23 19:04:13 +00:00
// Check on seal status and storage type before proceeding:
// If raft is used for storage, core needs to be sealed
if ! isRaftHAOnly && ! c . Sealed ( ) {
2021-03-03 18:59:50 +00:00
c . logger . Error ( "node must be sealed before joining" )
2020-06-23 19:04:13 +00:00
return false , errors . New ( "node must be sealed before joining" )
}
// If raft is used for ha-only, core needs to be unsealed
if isRaftHAOnly && c . Sealed ( ) {
c . logger . Error ( "node must be unsealed before joining" )
return false , errors . New ( "node must be unsealed before joining" )
}
2020-10-13 20:26:39 +00:00
// Disallow leader API address to be provided if we're using raft for HA-only.
2020-06-23 19:04:13 +00:00
// The leader API address is obtained directly through storage. This serves
// as a form of verification that this node is sharing the same physical
// storage as the leader node.
if isRaftHAOnly {
for _ , info := range leaderInfos {
2020-10-13 20:26:39 +00:00
if info . LeaderAPIAddr != "" || info . AutoJoin != "" {
return false , errors . New ( "leader API address and auto-join metadata must be unset when raft is used exclusively for HA" )
2020-06-23 19:04:13 +00:00
}
}
// Get the leader address from storage
keys , err := c . barrier . List ( ctx , coreLeaderPrefix )
if err != nil {
return false , err
}
if len ( keys ) == 0 || len ( keys [ 0 ] ) == 0 {
return false , errors . New ( "unable to fetch leadership entry" )
}
leadershipEntry := coreLeaderPrefix + keys [ 0 ]
entry , err := c . barrier . Get ( ctx , leadershipEntry )
if err != nil {
return false , err
}
if entry == nil {
return false , errors . New ( "unable to read leadership entry" )
}
var adv activeAdvertisement
err = jsonutil . DecodeJSON ( entry . Value , & adv )
if err != nil {
2021-05-11 17:12:54 +00:00
return false , fmt . Errorf ( "unable to decoded leader entry: %w" , err )
2020-06-23 19:04:13 +00:00
}
leaderInfos [ 0 ] . LeaderAPIAddr = adv . RedirectAddr
}
2020-10-13 20:26:39 +00:00
disco , err := newDiscover ( )
if err != nil {
2021-05-11 17:12:54 +00:00
return false , fmt . Errorf ( "failed to create auto-join discovery: %w" , err )
2020-10-13 20:26:39 +00:00
}
2020-01-14 01:02:16 +00:00
join := func ( retry bool ) error {
2020-10-13 20:26:39 +00:00
joinLeader := func ( leaderInfo * raft . LeaderJoinInfo , leaderAddr string ) error {
2020-01-14 01:02:16 +00:00
if leaderInfo == nil {
return errors . New ( "raft leader information is nil" )
}
2020-10-13 20:26:39 +00:00
if len ( leaderAddr ) == 0 {
2020-01-14 01:02:16 +00:00
return errors . New ( "raft leader address not provided" )
}
2019-06-20 19:14:58 +00:00
2020-12-08 18:55:34 +00:00
init , err := c . InitializedLocally ( ctx )
2020-01-14 01:02:16 +00:00
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to check if core is initialized: %w" , err )
2020-01-14 01:02:16 +00:00
}
2020-06-23 19:04:13 +00:00
if init && ! isRaftHAOnly {
2020-01-14 01:02:16 +00:00
c . logger . Info ( "returning from raft join as the node is initialized" )
return nil
}
2019-06-20 19:14:58 +00:00
2020-10-13 20:26:39 +00:00
c . logger . Info ( "attempting to join possible raft leader node" , "leader_addr" , leaderAddr )
2019-06-20 19:14:58 +00:00
2020-01-14 01:02:16 +00:00
// Create an API client to interact with the leader node
transport := cleanhttp . DefaultPooledTransport ( )
2019-06-20 19:14:58 +00:00
2020-01-14 01:02:16 +00:00
if leaderInfo . TLSConfig == nil && ( len ( leaderInfo . LeaderCACert ) != 0 || len ( leaderInfo . LeaderClientCert ) != 0 || len ( leaderInfo . LeaderClientKey ) != 0 ) {
leaderInfo . TLSConfig , err = tlsutil . ClientTLSConfig ( [ ] byte ( leaderInfo . LeaderCACert ) , [ ] byte ( leaderInfo . LeaderClientCert ) , [ ] byte ( leaderInfo . LeaderClientKey ) )
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to create TLS config: %w" , err )
2020-01-14 01:02:16 +00:00
}
2021-01-19 22:54:28 +00:00
leaderInfo . TLSConfig . ServerName = leaderInfo . LeaderTLSServerName
2020-01-14 01:02:16 +00:00
}
2021-04-06 13:16:54 +00:00
if leaderInfo . TLSConfig == nil && leaderInfo . LeaderTLSServerName != "" {
leaderInfo . TLSConfig , err = tlsutil . SetupTLSConfig ( map [ string ] string { "address" : leaderInfo . LeaderTLSServerName } , "" )
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to create TLS config: %w" , err )
2021-04-06 13:16:54 +00:00
}
}
2019-06-20 19:14:58 +00:00
2020-01-14 01:02:16 +00:00
if leaderInfo . TLSConfig != nil {
transport . TLSClientConfig = leaderInfo . TLSConfig . Clone ( )
if err := http2 . ConfigureTransport ( transport ) ; err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to configure TLS: %w" , err )
2020-01-14 01:02:16 +00:00
}
}
client := & http . Client {
Transport : transport ,
}
2020-10-13 20:26:39 +00:00
2020-01-14 01:02:16 +00:00
config := api . DefaultConfig ( )
if config . Error != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to create api client: %w" , config . Error )
2020-01-14 01:02:16 +00:00
}
2020-10-13 20:26:39 +00:00
config . Address = leaderAddr
2020-01-14 01:02:16 +00:00
config . HttpClient = client
config . MaxRetries = 0
2020-10-13 20:26:39 +00:00
2020-01-14 01:02:16 +00:00
apiClient , err := api . NewClient ( config )
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to create api client: %w" , err )
2020-01-14 01:02:16 +00:00
}
// Attempt to join the leader by requesting for the bootstrap challenge
secret , err := apiClient . Logical ( ) . Write ( "sys/storage/raft/bootstrap/challenge" , map [ string ] interface { } {
2020-06-23 19:04:13 +00:00
"server_id" : raftBackend . NodeID ( ) ,
2020-01-14 01:02:16 +00:00
} )
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "error during raft bootstrap init call: %w" , err )
2020-01-14 01:02:16 +00:00
}
if secret == nil {
return errors . New ( "could not retrieve raft bootstrap package" )
}
var sealConfig SealConfig
err = mapstructure . Decode ( secret . Data [ "seal_config" ] , & sealConfig )
if err != nil {
return err
}
if sealConfig . Type != c . seal . BarrierType ( ) {
return fmt . Errorf ( "mismatching seal types between raft leader (%s) and follower (%s)" , sealConfig . Type , c . seal . BarrierType ( ) )
}
challengeB64 , ok := secret . Data [ "challenge" ]
if ! ok {
return errors . New ( "error during raft bootstrap call, no challenge given" )
}
challengeRaw , err := base64 . StdEncoding . DecodeString ( challengeB64 . ( string ) )
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "error decoding raft bootstrap challenge: %w" , err )
2020-01-14 01:02:16 +00:00
}
eBlob := & wrapping . EncryptedBlobInfo { }
if err := proto . Unmarshal ( challengeRaw , eBlob ) ; err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "error decoding raft bootstrap challenge: %w" , err )
2020-01-14 01:02:16 +00:00
}
2020-10-13 20:26:39 +00:00
2020-01-14 01:02:16 +00:00
raftInfo := & raftInformation {
challenge : eBlob ,
leaderClient : apiClient ,
leaderBarrierConfig : & sealConfig ,
2021-02-10 21:41:58 +00:00
nonVoter : nonVoter ,
2020-01-14 01:02:16 +00:00
}
2020-06-23 19:04:13 +00:00
// If we're using Shamir and using raft for both physical and HA, we
// need to block until the node is unsealed, unless retry is set to
// false.
if c . seal . BarrierType ( ) == wrapping . Shamir && ! isRaftHAOnly {
2020-01-14 01:02:16 +00:00
c . raftInfo = raftInfo
if err := c . seal . SetBarrierConfig ( ctx , & sealConfig ) ; err != nil {
return err
}
if ! retry {
return nil
}
// Wait until unseal keys are supplied
c . raftInfo . joinInProgress = true
if atomic . LoadUint32 ( c . postUnsealStarted ) != 1 {
return errors . New ( "waiting for unseal keys to be supplied" )
}
}
if err := c . joinRaftSendAnswer ( ctx , c . seal . GetAccess ( ) , raftInfo ) ; err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to send answer to raft leader node: %w" , err )
2020-01-14 01:02:16 +00:00
}
2020-06-23 19:04:13 +00:00
if c . seal . BarrierType ( ) == wrapping . Shamir && ! isRaftHAOnly {
2020-01-14 01:02:16 +00:00
// Reset the state
c . raftInfo = nil
// In case of Shamir unsealing, inform the unseal process that raft join is completed
close ( c . raftJoinDoneCh )
}
c . logger . Info ( "successfully joined the raft cluster" , "leader_addr" , leaderInfo . LeaderAPIAddr )
2019-06-20 19:14:58 +00:00
return nil
}
2020-01-14 01:02:16 +00:00
// Each join try goes through all the possible leader nodes and attempts to join
// them, until one of the attempt succeeds.
for _ , leaderInfo := range leaderInfos {
2020-10-13 20:26:39 +00:00
switch {
case leaderInfo . LeaderAPIAddr != "" && leaderInfo . AutoJoin != "" :
2020-10-23 20:13:09 +00:00
c . logger . Error ( "join attempt failed" , "error" , errors . New ( "cannot provide both leader address and auto-join metadata" ) )
2020-10-13 20:26:39 +00:00
case leaderInfo . LeaderAPIAddr != "" :
if err := joinLeader ( leaderInfo , leaderInfo . LeaderAPIAddr ) ; err != nil {
2020-10-23 20:13:09 +00:00
c . logger . Warn ( "join attempt failed" , "error" , err )
2020-10-13 20:26:39 +00:00
} else {
// successfully joined leader
return nil
}
case leaderInfo . AutoJoin != "" :
addrs , err := disco . Addrs ( leaderInfo . AutoJoin , c . logger . StandardLogger ( nil ) )
if err != nil {
2020-10-23 20:13:09 +00:00
c . logger . Error ( "failed to parse addresses from auto-join metadata" , "error" , err )
2020-10-13 20:26:39 +00:00
}
for _ , addr := range addrs {
2020-10-23 20:13:09 +00:00
u , err := url . Parse ( addr )
if err != nil {
c . logger . Error ( "failed to parse discovered address" , "error" , err )
continue
}
if u . Scheme == "" {
scheme := leaderInfo . AutoJoinScheme
if scheme == "" {
// default to HTTPS when no scheme is provided
scheme = "https"
}
addr = fmt . Sprintf ( "%s://%s" , scheme , addr )
}
if u . Port ( ) == "" {
port := leaderInfo . AutoJoinPort
if port == 0 {
// default to 8200 when no port is provided
port = 8200
}
addr = fmt . Sprintf ( "%s:%d" , addr , port )
}
2020-10-13 20:26:39 +00:00
if err := joinLeader ( leaderInfo , addr ) ; err != nil {
2020-10-23 20:13:09 +00:00
c . logger . Warn ( "join attempt failed" , "error" , err )
2020-10-13 20:26:39 +00:00
} else {
// successfully joined leader
return nil
}
}
default :
2020-10-23 20:13:09 +00:00
c . logger . Error ( "join attempt failed" , "error" , errors . New ( "must provide leader address or auto-join metadata" ) )
2020-01-14 01:02:16 +00:00
}
2019-06-20 19:14:58 +00:00
}
2020-01-14 01:02:16 +00:00
return errors . New ( "failed to join any raft leader node" )
2019-06-20 19:14:58 +00:00
}
2020-01-14 01:02:16 +00:00
switch leaderInfos [ 0 ] . Retry {
2019-06-20 19:14:58 +00:00
case true :
go func ( ) {
for {
2020-01-14 01:02:16 +00:00
select {
case <- ctx . Done ( ) :
return
default :
}
err := join ( true )
2019-06-20 19:14:58 +00:00
if err == nil {
return
}
2020-01-14 01:02:16 +00:00
c . logger . Error ( "failed to retry join raft cluster" , "retry" , "2s" )
time . Sleep ( 2 * time . Second )
2019-06-20 19:14:58 +00:00
}
} ( )
// Backgrounded so return false
return false , nil
default :
2020-01-14 01:02:16 +00:00
if err := join ( false ) ; err != nil {
2019-06-20 19:14:58 +00:00
c . logger . Error ( "failed to join raft cluster" , "error" , err )
2021-05-11 17:12:54 +00:00
return false , fmt . Errorf ( "failed to join raft cluster: %w" , err )
2019-06-20 19:14:58 +00:00
}
}
return true , nil
}
2020-06-23 19:04:13 +00:00
// getRaftBackend returns the RaftBackend from the HA or physical backend,
// in that order of preference, or nil if not of type RaftBackend.
func ( c * Core ) getRaftBackend ( ) * raft . RaftBackend {
var raftBackend * raft . RaftBackend
if raftHA , ok := c . ha . ( * raft . RaftBackend ) ; ok {
raftBackend = raftHA
}
if raftStorage , ok := c . underlyingPhysical . ( * raft . RaftBackend ) ; ok {
raftBackend = raftStorage
}
return raftBackend
}
// isRaftHAOnly returns true if c.ha is raft and physical storage is non-raft
func ( c * Core ) isRaftHAOnly ( ) bool {
_ , isRaftHA := c . ha . ( * raft . RaftBackend )
_ , isRaftStorage := c . underlyingPhysical . ( * raft . RaftBackend )
return isRaftHA && ! isRaftStorage
}
2019-06-20 19:14:58 +00:00
2020-01-11 01:39:52 +00:00
func ( c * Core ) joinRaftSendAnswer ( ctx context . Context , sealAccess * seal . Access , raftInfo * raftInformation ) error {
2019-10-11 18:56:59 +00:00
if raftInfo . challenge == nil {
2019-06-20 19:14:58 +00:00
return errors . New ( "raft challenge is nil" )
}
2020-06-23 19:04:13 +00:00
raftBackend := c . getRaftBackend ( )
if raftBackend == nil {
return errors . New ( "raft backend is not in use" )
2019-06-20 19:14:58 +00:00
}
2020-06-23 19:04:13 +00:00
if raftBackend . Initialized ( ) {
2019-06-20 19:14:58 +00:00
return errors . New ( "raft is already initialized" )
}
2020-01-11 01:39:52 +00:00
plaintext , err := sealAccess . Decrypt ( ctx , raftInfo . challenge , nil )
2019-06-20 19:14:58 +00:00
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "error decrypting challenge: %w" , err )
2019-06-20 19:14:58 +00:00
}
2019-06-27 17:00:03 +00:00
parsedClusterAddr , err := url . Parse ( c . ClusterAddr ( ) )
2019-06-20 19:14:58 +00:00
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "error parsing cluster address: %w" , err )
2019-06-20 19:14:58 +00:00
}
clusterAddr := parsedClusterAddr . Host
2020-06-23 19:04:13 +00:00
if atomic . LoadUint32 ( & TestingUpdateClusterAddr ) == 1 && strings . HasSuffix ( clusterAddr , ":0" ) {
2019-06-20 19:14:58 +00:00
// We are testing and have an address provider, so just create a random
// addr, it will be overwritten later.
var err error
clusterAddr , err = uuid . GenerateUUID ( )
if err != nil {
return err
}
}
2019-10-11 18:56:59 +00:00
answerReq := raftInfo . leaderClient . NewRequest ( "PUT" , "/v1/sys/storage/raft/bootstrap/answer" )
2019-06-20 19:14:58 +00:00
if err := answerReq . SetJSONBody ( map [ string ] interface { } {
"answer" : base64 . StdEncoding . EncodeToString ( plaintext ) ,
"cluster_addr" : clusterAddr ,
2020-06-23 19:04:13 +00:00
"server_id" : raftBackend . NodeID ( ) ,
2021-02-10 21:41:58 +00:00
"non_voter" : raftInfo . nonVoter ,
2019-06-20 19:14:58 +00:00
} ) ; err != nil {
return err
}
2019-10-11 18:56:59 +00:00
answerRespJson , err := raftInfo . leaderClient . RawRequestWithContext ( ctx , answerReq )
2019-06-20 19:14:58 +00:00
if answerRespJson != nil {
defer answerRespJson . Body . Close ( )
}
if err != nil {
return err
}
var answerResp answerRespData
if err := jsonutil . DecodeJSONFromReader ( answerRespJson . Body , & answerResp ) ; err != nil {
return err
}
2021-05-19 20:07:58 +00:00
if answerResp . Data . AutoloadedLicense && ! LicenseAutoloaded ( c ) {
return ErrJoinWithoutAutoloading
}
2020-06-23 19:04:13 +00:00
if err := raftBackend . Bootstrap ( answerResp . Data . Peers ) ; err != nil {
return err
}
2019-06-20 19:14:58 +00:00
err = c . startClusterListener ( ctx )
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "error starting cluster: %w" , err )
2019-06-20 19:14:58 +00:00
}
2020-06-23 19:04:13 +00:00
raftBackend . SetRestoreCallback ( c . raftSnapshotRestoreCallback ( true , true ) )
2021-03-03 18:59:50 +00:00
opts := raft . SetupOpts {
2019-07-29 20:05:43 +00:00
TLSKeyring : answerResp . Data . TLSKeyring ,
2019-09-03 15:59:56 +00:00
ClusterListener : c . getClusterListener ( ) ,
2021-03-03 18:59:50 +00:00
}
err = raftBackend . SetupCluster ( ctx , opts )
2019-06-20 19:14:58 +00:00
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "failed to setup raft cluster: %w" , err )
2019-06-20 19:14:58 +00:00
}
return nil
}
2021-03-03 18:59:50 +00:00
func ( c * Core ) loadAutopilotConfiguration ( ctx context . Context ) ( * raft . AutopilotConfig , error ) {
var autopilotConfig * raft . AutopilotConfig
entry , err := c . barrier . Get ( ctx , raftAutopilotConfigurationStoragePath )
if err != nil {
return nil , err
}
if entry == nil {
return nil , nil
}
if err := jsonutil . DecodeJSON ( entry . Value , & autopilotConfig ) ; err != nil {
return nil , err
}
return autopilotConfig , nil
}
2020-06-23 19:04:13 +00:00
// RaftBootstrap performs bootstrapping of a raft cluster if core contains a raft
// backend. If raft is not part for the storage or HA storage backend, this
// call results in an error.
func ( c * Core ) RaftBootstrap ( ctx context . Context , onInit bool ) error {
if c . logger . IsDebug ( ) {
c . logger . Debug ( "bootstrapping raft backend" )
defer c . logger . Debug ( "finished bootstrapping raft backend" )
}
raftBackend := c . getRaftBackend ( )
if raftBackend == nil {
return errors . New ( "raft backend not in use" )
}
parsedClusterAddr , err := url . Parse ( c . ClusterAddr ( ) )
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "error parsing cluster address: %w" , err )
2020-06-23 19:04:13 +00:00
}
if err := raftBackend . Bootstrap ( [ ] raft . Peer {
{
ID : raftBackend . NodeID ( ) ,
Address : parsedClusterAddr . Host ,
} ,
} ) ; err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "could not bootstrap clustered storage: %w" , err )
2020-06-23 19:04:13 +00:00
}
raftOpts := raft . SetupOpts {
StartAsLeader : true ,
}
if ! onInit {
// Generate the TLS Keyring info for SetupCluster to consume
raftTLS , err := c . raftCreateTLSKeyring ( ctx )
if err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "could not generate TLS keyring during bootstrap: %w" , err )
2020-06-23 19:04:13 +00:00
}
raftBackend . SetRestoreCallback ( c . raftSnapshotRestoreCallback ( true , true ) )
raftOpts . ClusterListener = c . getClusterListener ( )
raftOpts . TLSKeyring = raftTLS
}
if err := raftBackend . SetupCluster ( ctx , raftOpts ) ; err != nil {
2021-05-11 17:12:54 +00:00
return fmt . Errorf ( "could not start clustered storage: %w" , err )
2020-06-23 19:04:13 +00:00
}
return nil
}
2019-06-20 19:14:58 +00:00
func ( c * Core ) isRaftUnseal ( ) bool {
2019-10-11 18:56:59 +00:00
return c . raftInfo != nil
2019-06-20 19:14:58 +00:00
}
type answerRespData struct {
Data answerResp ` json:"data" `
}
type answerResp struct {
2021-05-19 20:07:58 +00:00
Peers [ ] raft . Peer ` json:"peers" `
TLSKeyring * raft . TLSKeyring ` json:"tls_keyring" `
AutoloadedLicense bool ` json:"autoloaded_license" `
2019-06-20 19:14:58 +00:00
}
2020-10-13 20:26:39 +00:00
func newDiscover ( ) ( * discover . Discover , error ) {
providers := make ( map [ string ] discover . Provider )
for k , v := range discover . Providers {
providers [ k ] = v
}
2020-10-23 20:13:09 +00:00
providers [ "k8s" ] = & discoverk8s . Provider { }
2020-10-13 20:26:39 +00:00
return discover . New (
discover . WithProviders ( providers ) ,
)
}