mirror of
				https://github.com/optim-enterprises-bv/vault.git
				synced 2025-10-30 18:17:55 +00:00 
			
		
		
		
	VAULT-14048: raft-autopilot appears to refuse to remove a node which has left and wouldn't impact stability (#19472)
* ensure we supply the node type when it's for a voter * bumped autopilot version back to v0.2.0 and ran go mod tidy * changed condition in knownservers and added some comments * Export GetRaftBackend * Updated tests for autopilot (related to dead server cleanup) * Export Raft NewDelegate Co-authored-by: Nick Cabatoff <ncabatoff@hashicorp.com>
This commit is contained in:
		
							
								
								
									
										3
									
								
								changelog/19472.txt
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										3
									
								
								changelog/19472.txt
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,3 @@ | ||||
| ```release-note:improvement | ||||
| autopilot: Update version to v0.2.0 to add better support for respecting min quorum | ||||
| ``` | ||||
							
								
								
									
										2
									
								
								go.mod
									
									
									
									
									
								
							
							
						
						
									
										2
									
								
								go.mod
									
									
									
									
									
								
							| @@ -107,7 +107,7 @@ require ( | ||||
| 	github.com/hashicorp/hcp-sdk-go v0.23.0 | ||||
| 	github.com/hashicorp/nomad/api v0.0.0-20220707195938-75f4c2237b28 | ||||
| 	github.com/hashicorp/raft v1.3.10 | ||||
| 	github.com/hashicorp/raft-autopilot v0.1.6 | ||||
| 	github.com/hashicorp/raft-autopilot v0.2.0 | ||||
| 	github.com/hashicorp/raft-boltdb/v2 v2.0.0-20210421194847-a7e34179d62c | ||||
| 	github.com/hashicorp/raft-snapshot v1.0.4 | ||||
| 	github.com/hashicorp/vault-plugin-auth-alicloud v0.14.0 | ||||
|   | ||||
							
								
								
									
										4
									
								
								go.sum
									
									
									
									
									
								
							
							
						
						
									
										4
									
								
								go.sum
									
									
									
									
									
								
							| @@ -939,8 +939,8 @@ github.com/hashicorp/raft v1.1.2-0.20191002163536-9c6bd3e3eb17/go.mod h1:vPAJM8A | ||||
| github.com/hashicorp/raft v1.2.0/go.mod h1:vPAJM8Asw6u8LxC3eJCUZmRP/E4QmUGE1R7g7k8sG/8= | ||||
| github.com/hashicorp/raft v1.3.10 h1:LR5QZX1VQd0DFWZfeCwWawyeKfpS/Tm1yjnJIY5X4Tw= | ||||
| github.com/hashicorp/raft v1.3.10/go.mod h1:J8naEwc6XaaCfts7+28whSeRvCqTd6e20BlCU3LtEO4= | ||||
| github.com/hashicorp/raft-autopilot v0.1.6 h1:C1q3RNF2FfXNZfHWbvVAu0QixaQK8K5pX4O5lh+9z4I= | ||||
| github.com/hashicorp/raft-autopilot v0.1.6/go.mod h1:Af4jZBwaNOI+tXfIqIdbcAnh/UyyqIMj/pOISIfhArw= | ||||
| github.com/hashicorp/raft-autopilot v0.2.0 h1:2/R2RPgamgRKgNWGQioULZvjeKXQZmDuw5Ty+6c+H7Y= | ||||
| github.com/hashicorp/raft-autopilot v0.2.0/go.mod h1:q6tZ8UAZ5xio2gv2JvjgmtOlh80M6ic8xQYBe2Egkg8= | ||||
| github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea/go.mod h1:pNv7Wc3ycL6F5oOWn+tPGo2gWD4a5X+yp/ntwdKLjRk= | ||||
| github.com/hashicorp/raft-boltdb/v2 v2.0.0-20210421194847-a7e34179d62c h1:oiKun9QlrOz5yQxMZJ3tf1kWtFYuKSJzxzEDxDPevj4= | ||||
| github.com/hashicorp/raft-boltdb/v2 v2.0.0-20210421194847-a7e34179d62c/go.mod h1:kiPs9g148eLShc2TYagUAyKDnD+dH9U+CQKsXzlY9xo= | ||||
|   | ||||
| @@ -20,3 +20,34 @@ func WaitForActiveNodeAndStandbys(t testing.T, cluster *vault.TestCluster) { | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
|  | ||||
| // WaitForNodesExcludingSelectedStandbys is variation on WaitForActiveNodeAndStandbys. | ||||
| // It waits for the active node before waiting for standby nodes, however | ||||
| // it will not wait for cores with indexes that match those specified as arguments. | ||||
| // Whilst you could specify index 0 which is likely to be the leader node, the function | ||||
| // checks for the leader first regardless of the indexes to skip, so it would be redundant to do so. | ||||
| // The intention/use case for this function is to allow a cluster to start and become active with one | ||||
| // or more nodes not joined, so that we can test scenarios where a node joins later. | ||||
| // e.g. 4 nodes in the cluster, only 3 nodes in cluster 'active', 1 node can be joined later in tests. | ||||
| func WaitForNodesExcludingSelectedStandbys(t testing.T, cluster *vault.TestCluster, indexesToSkip ...int) { | ||||
| 	WaitForActiveNode(t, cluster) | ||||
|  | ||||
| 	contains := func(elems []int, e int) bool { | ||||
| 		for _, v := range elems { | ||||
| 			if v == e { | ||||
| 				return true | ||||
| 			} | ||||
| 		} | ||||
|  | ||||
| 		return false | ||||
| 	} | ||||
| 	for i, core := range cluster.Cores { | ||||
| 		if contains(indexesToSkip, i) { | ||||
| 			continue | ||||
| 		} | ||||
|  | ||||
| 		if standby, _ := core.Core.Standby(); standby { | ||||
| 			WaitForStandbyNode(t, core) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
|   | ||||
| @@ -292,7 +292,7 @@ type Delegate struct { | ||||
| 	emptyVersionLogs map[raft.ServerID]struct{} | ||||
| } | ||||
|  | ||||
| func newDelegate(b *RaftBackend) *Delegate { | ||||
| func NewDelegate(b *RaftBackend) *Delegate { | ||||
| 	return &Delegate{ | ||||
| 		RaftBackend:      b, | ||||
| 		inflightRemovals: make(map[raft.ServerID]bool), | ||||
| @@ -385,6 +385,7 @@ func (d *Delegate) KnownServers() map[raft.ServerID]*autopilot.Server { | ||||
| 		return nil | ||||
| 	} | ||||
|  | ||||
| 	apServerStates := d.autopilot.GetState().Servers | ||||
| 	servers := future.Configuration().Servers | ||||
| 	serverIDs := make([]string, 0, len(servers)) | ||||
| 	for _, server := range servers { | ||||
| @@ -428,6 +429,19 @@ func (d *Delegate) KnownServers() map[raft.ServerID]*autopilot.Server { | ||||
| 			Ext:         d.autopilotServerExt(state), | ||||
| 		} | ||||
|  | ||||
| 		// As KnownServers is a delegate called by autopilot let's check if we already | ||||
| 		// had this data in the correct format and use it. If we don't (which sounds a | ||||
| 		// bit sad, unless this ISN'T a voter) then as a fail-safe, let's try what we've | ||||
| 		// done elsewhere in code to check the desired suffrage and manually set NodeType | ||||
| 		// based on whether that's a voter or not. If we don't  do either of these | ||||
| 		// things, NodeType isn't set which means technically it's not a voter. | ||||
| 		// It shouldn't be a voter and end up in this state. | ||||
| 		if apServerState, found := apServerStates[raft.ServerID(id)]; found && apServerState.Server.NodeType != "" { | ||||
| 			server.NodeType = apServerState.Server.NodeType | ||||
| 		} else if state.DesiredSuffrage == "voter" { | ||||
| 			server.NodeType = autopilot.NodeVoter | ||||
| 		} | ||||
|  | ||||
| 		switch state.IsDead.Load() { | ||||
| 		case true: | ||||
| 			d.logger.Debug("informing autopilot that the node left", "id", id) | ||||
| @@ -445,6 +459,7 @@ func (d *Delegate) KnownServers() map[raft.ServerID]*autopilot.Server { | ||||
| 		Name:        d.localID, | ||||
| 		RaftVersion: raft.ProtocolVersionMax, | ||||
| 		NodeStatus:  autopilot.NodeAlive, | ||||
| 		NodeType:    autopilot.NodeVoter, // The leader must be a voter | ||||
| 		Meta: d.meta(&FollowerState{ | ||||
| 			UpgradeVersion: d.EffectiveVersion(), | ||||
| 			RedundancyZone: d.RedundancyZone(), | ||||
| @@ -820,7 +835,7 @@ func (b *RaftBackend) SetupAutopilot(ctx context.Context, storageConfig *Autopil | ||||
| 	if b.autopilotUpdateInterval != 0 { | ||||
| 		options = append(options, autopilot.WithUpdateInterval(b.autopilotUpdateInterval)) | ||||
| 	} | ||||
| 	b.autopilot = autopilot.New(b.raft, newDelegate(b), options...) | ||||
| 	b.autopilot = autopilot.New(b.raft, NewDelegate(b), options...) | ||||
| 	b.followerStates = followerStates | ||||
| 	b.followerHeartbeatTicker = time.NewTicker(1 * time.Second) | ||||
|  | ||||
|   | ||||
| @@ -5,9 +5,11 @@ package rafttests | ||||
|  | ||||
| import ( | ||||
| 	"context" | ||||
| 	"encoding/json" | ||||
| 	"fmt" | ||||
| 	"math" | ||||
| 	"reflect" | ||||
| 	"sync/atomic" | ||||
| 	"testing" | ||||
| 	"time" | ||||
|  | ||||
| @@ -258,8 +260,8 @@ func TestRaft_Autopilot_Stabilization_Delay(t *testing.T) { | ||||
| 		} | ||||
| 	} | ||||
|  | ||||
| 	join(t, cluster.Cores[1], client, cluster) | ||||
| 	join(t, cluster.Cores[2], client, cluster) | ||||
| 	joinAndUnseal(t, cluster.Cores[1], cluster, false, false) | ||||
| 	joinAndUnseal(t, cluster.Cores[2], cluster, false, false) | ||||
|  | ||||
| 	core2shouldBeHealthyAt := time.Now().Add(timeToHealthyCore2) | ||||
|  | ||||
| @@ -320,8 +322,8 @@ func TestRaft_AutoPilot_Peersets_Equivalent(t *testing.T) { | ||||
| 	}) | ||||
| 	require.NoError(t, err) | ||||
|  | ||||
| 	join(t, cluster.Cores[1], client, cluster) | ||||
| 	join(t, cluster.Cores[2], client, cluster) | ||||
| 	joinAsVoterAndUnseal(t, cluster.Cores[1], cluster) | ||||
| 	joinAsVoterAndUnseal(t, cluster.Cores[2], cluster) | ||||
|  | ||||
| 	deadline := time.Now().Add(10 * time.Second) | ||||
| 	var core0Peers, core1Peers, core2Peers []raft.Peer | ||||
| @@ -349,74 +351,6 @@ func TestRaft_AutoPilot_Peersets_Equivalent(t *testing.T) { | ||||
| 	require.Equal(t, core1Peers, core2Peers) | ||||
| } | ||||
|  | ||||
| func joinAndStabilizeAndPromote(t *testing.T, core *vault.TestClusterCore, client *api.Client, cluster *vault.TestCluster, config *api.AutopilotConfig, nodeID string, numServers int) { | ||||
| 	joinAndStabilize(t, core, client, cluster, config, nodeID, numServers) | ||||
|  | ||||
| 	// Now that the server is stable, wait for autopilot to reconcile and | ||||
| 	// promotion to happen. Reconcile interval is 10 seconds. Bound it by | ||||
| 	// doubling. | ||||
| 	deadline := time.Now().Add(2 * autopilot.DefaultReconcileInterval) | ||||
| 	failed := true | ||||
| 	var err error | ||||
| 	var state *api.AutopilotState | ||||
| 	for time.Now().Before(deadline) { | ||||
| 		state, err = client.Sys().RaftAutopilotState() | ||||
| 		require.NoError(t, err) | ||||
| 		if state.Servers[nodeID].Status == "voter" { | ||||
| 			failed = false | ||||
| 			break | ||||
| 		} | ||||
| 		time.Sleep(1 * time.Second) | ||||
| 	} | ||||
|  | ||||
| 	if failed { | ||||
| 		t.Fatalf("autopilot failed to promote node: id: %#v: state:%# v\n", nodeID, pretty.Formatter(state)) | ||||
| 	} | ||||
| } | ||||
|  | ||||
| func joinAndStabilize(t *testing.T, core *vault.TestClusterCore, client *api.Client, cluster *vault.TestCluster, config *api.AutopilotConfig, nodeID string, numServers int) { | ||||
| 	t.Helper() | ||||
| 	join(t, core, client, cluster) | ||||
| 	time.Sleep(2 * time.Second) | ||||
|  | ||||
| 	state, err := client.Sys().RaftAutopilotState() | ||||
| 	require.NoError(t, err) | ||||
| 	require.Equal(t, false, state.Healthy) | ||||
| 	require.Len(t, state.Servers, numServers) | ||||
| 	require.Equal(t, false, state.Servers[nodeID].Healthy) | ||||
| 	require.Equal(t, "alive", state.Servers[nodeID].NodeStatus) | ||||
| 	require.Equal(t, "non-voter", state.Servers[nodeID].Status) | ||||
|  | ||||
| 	// Wait till the stabilization period is over | ||||
| 	deadline := time.Now().Add(config.ServerStabilizationTime) | ||||
| 	healthy := false | ||||
| 	for time.Now().Before(deadline) { | ||||
| 		state, err := client.Sys().RaftAutopilotState() | ||||
| 		require.NoError(t, err) | ||||
| 		if state.Healthy { | ||||
| 			healthy = true | ||||
| 		} | ||||
| 		time.Sleep(1 * time.Second) | ||||
| 	} | ||||
| 	if !healthy { | ||||
| 		t.Fatalf("cluster failed to stabilize") | ||||
| 	} | ||||
| } | ||||
|  | ||||
| func join(t *testing.T, core *vault.TestClusterCore, client *api.Client, cluster *vault.TestCluster) { | ||||
| 	t.Helper() | ||||
| 	_, err := core.JoinRaftCluster(namespace.RootContext(context.Background()), []*raft.LeaderJoinInfo{ | ||||
| 		{ | ||||
| 			LeaderAPIAddr: client.Address(), | ||||
| 			TLSConfig:     cluster.Cores[0].TLSConfig(), | ||||
| 			Retry:         true, | ||||
| 		}, | ||||
| 	}, false) | ||||
| 	require.NoError(t, err) | ||||
| 	time.Sleep(1 * time.Second) | ||||
| 	cluster.UnsealCore(t, core) | ||||
| } | ||||
|  | ||||
| // TestRaft_VotersStayVoters ensures that autopilot doesn't demote a node just | ||||
| // because it hasn't been heard from in some time. | ||||
| func TestRaft_VotersStayVoters(t *testing.T) { | ||||
| @@ -464,7 +398,230 @@ func TestRaft_VotersStayVoters(t *testing.T) { | ||||
| 	// new leader won't have seen any heartbeats initially - and create a "down" | ||||
| 	// node that won't be sending heartbeats. | ||||
| 	testhelpers.EnsureCoreSealed(t, cluster.Cores[0]) | ||||
| 	time.Sleep(30 * time.Second) | ||||
| 	time.Sleep(config.ServerStabilizationTime + 2*time.Second) | ||||
| 	client = cluster.Cores[1].Client | ||||
| 	errIfNonVotersExist() | ||||
| 	err = errIfNonVotersExist() | ||||
| 	require.NoError(t, err) | ||||
| } | ||||
|  | ||||
| // TestRaft_Autopilot_DeadServerCleanup tests that dead servers are correctly | ||||
| // removed by Vault and autopilot when a node stops and a replacement node joins. | ||||
| // The expected behavior is that removing a node from a 3 node cluster wouldn't | ||||
| // remove it from Raft until a replacement voter had joined and stabilized/been promoted. | ||||
| func TestRaft_Autopilot_DeadServerCleanup(t *testing.T) { | ||||
| 	conf, opts := teststorage.ClusterSetup(nil, nil, teststorage.RaftBackendSetup) | ||||
| 	conf.DisableAutopilot = false | ||||
| 	opts.NumCores = 4 | ||||
| 	opts.SetupFunc = nil | ||||
| 	opts.PhysicalFactoryConfig = map[string]interface{}{ | ||||
| 		"autopilot_reconcile_interval": "300ms", | ||||
| 		"autopilot_update_interval":    "100ms", | ||||
| 	} | ||||
|  | ||||
| 	cluster := vault.NewTestCluster(t, conf, opts) | ||||
| 	cluster.Start() | ||||
| 	defer cluster.Cleanup() | ||||
| 	leader, addressProvider := setupLeaderAndUnseal(t, cluster) | ||||
|  | ||||
| 	// Join 2 extra nodes manually, store the 3rd for later | ||||
| 	core1 := cluster.Cores[1] | ||||
| 	core2 := cluster.Cores[2] | ||||
| 	core3 := cluster.Cores[3] | ||||
| 	core1.UnderlyingRawStorage.(*raft.RaftBackend).SetServerAddressProvider(addressProvider) | ||||
| 	core2.UnderlyingRawStorage.(*raft.RaftBackend).SetServerAddressProvider(addressProvider) | ||||
| 	core3.UnderlyingRawStorage.(*raft.RaftBackend).SetServerAddressProvider(addressProvider) | ||||
| 	joinAsVoterAndUnseal(t, core1, cluster) | ||||
| 	joinAsVoterAndUnseal(t, core2, cluster) | ||||
| 	// Do not join node 3 | ||||
| 	testhelpers.WaitForNodesExcludingSelectedStandbys(t, cluster, 3) | ||||
|  | ||||
| 	config, err := leader.Client.Sys().RaftAutopilotConfiguration() | ||||
| 	require.NoError(t, err) | ||||
| 	require.True(t, isHealthyAfterStabilization(t, leader, config.ServerStabilizationTime)) | ||||
|  | ||||
| 	// Ensure Autopilot has the aggressive settings | ||||
| 	config.CleanupDeadServers = true | ||||
| 	config.ServerStabilizationTime = 5 * time.Second | ||||
| 	config.DeadServerLastContactThreshold = 10 * time.Second | ||||
| 	config.MaxTrailingLogs = 10 | ||||
| 	config.LastContactThreshold = 10 * time.Second | ||||
| 	config.MinQuorum = 3 | ||||
|  | ||||
| 	// We can't use Client.Sys().PutRaftAutopilotConfiguration(config) in OSS as disable_upgrade_migration isn't in OSS | ||||
| 	b, err := json.Marshal(&config) | ||||
| 	require.NoError(t, err) | ||||
| 	var m map[string]interface{} | ||||
| 	err = json.Unmarshal(b, &m) | ||||
| 	require.NoError(t, err) | ||||
| 	delete(m, "disable_upgrade_migration") | ||||
| 	_, err = leader.Client.Logical().Write("sys/storage/raft/autopilot/configuration", m) | ||||
| 	require.NoError(t, err) | ||||
|  | ||||
| 	// Observe for healthy state | ||||
| 	state, err := leader.Client.Sys().RaftAutopilotState() | ||||
| 	require.NoError(t, err) | ||||
| 	require.True(t, state.Healthy) | ||||
|  | ||||
| 	// Kill a node (core-2) | ||||
| 	cluster.StopCore(t, 2) | ||||
| 	// Wait for just over the dead server threshold to ensure the core is classed as 'dead' | ||||
| 	time.Sleep(config.DeadServerLastContactThreshold + 2*time.Second) | ||||
|  | ||||
| 	// Observe for an unhealthy state (but we still have 3 voters according to Raft) | ||||
| 	state, err = leader.Client.Sys().RaftAutopilotState() | ||||
| 	require.NoError(t, err) | ||||
| 	require.False(t, state.Healthy) | ||||
| 	require.Len(t, state.Voters, 3) | ||||
|  | ||||
| 	// Join node 3 now | ||||
| 	joinAsVoterAndUnseal(t, core3, cluster) | ||||
|  | ||||
| 	// Stabilization time | ||||
| 	require.True(t, isHealthyAfterStabilization(t, leader, config.ServerStabilizationTime)) | ||||
|  | ||||
| 	// Observe for healthy and contains 3 correct voters | ||||
| 	state, err = leader.Client.Sys().RaftAutopilotState() | ||||
| 	require.NoError(t, err) | ||||
| 	require.True(t, state.Healthy) | ||||
| 	require.Len(t, state.Voters, 3) | ||||
| 	require.Contains(t, state.Voters, "core-0") | ||||
| 	require.Contains(t, state.Voters, "core-1") | ||||
| 	require.NotContains(t, state.Voters, "core-2") | ||||
| 	require.Contains(t, state.Voters, "core-3") | ||||
| } | ||||
|  | ||||
| func joinAndStabilizeAndPromote(t *testing.T, core *vault.TestClusterCore, client *api.Client, cluster *vault.TestCluster, config *api.AutopilotConfig, nodeID string, numServers int) { | ||||
| 	joinAndStabilize(t, core, client, cluster, config, nodeID, numServers) | ||||
|  | ||||
| 	// Now that the server is stable, wait for autopilot to reconcile and | ||||
| 	// promotion to happen. Reconcile interval is 10 seconds. Bound it by | ||||
| 	// doubling. | ||||
| 	deadline := time.Now().Add(2 * autopilot.DefaultReconcileInterval) | ||||
| 	failed := true | ||||
| 	var err error | ||||
| 	var state *api.AutopilotState | ||||
| 	for time.Now().Before(deadline) { | ||||
| 		state, err = client.Sys().RaftAutopilotState() | ||||
| 		require.NoError(t, err) | ||||
| 		if state.Servers[nodeID].Status == "voter" { | ||||
| 			failed = false | ||||
| 			break | ||||
| 		} | ||||
| 		time.Sleep(1 * time.Second) | ||||
| 	} | ||||
|  | ||||
| 	if failed { | ||||
| 		t.Fatalf("autopilot failed to promote node: id: %#v: state:%# v\n", nodeID, pretty.Formatter(state)) | ||||
| 	} | ||||
| } | ||||
|  | ||||
| func joinAndStabilize(t *testing.T, core *vault.TestClusterCore, client *api.Client, cluster *vault.TestCluster, config *api.AutopilotConfig, nodeID string, numServers int) { | ||||
| 	t.Helper() | ||||
| 	joinAndUnseal(t, core, cluster, false, false) | ||||
| 	time.Sleep(2 * time.Second) | ||||
|  | ||||
| 	state, err := client.Sys().RaftAutopilotState() | ||||
| 	require.NoError(t, err) | ||||
| 	require.Equal(t, false, state.Healthy) | ||||
| 	require.Len(t, state.Servers, numServers) | ||||
| 	require.Equal(t, false, state.Servers[nodeID].Healthy) | ||||
| 	require.Equal(t, "alive", state.Servers[nodeID].NodeStatus) | ||||
| 	require.Equal(t, "non-voter", state.Servers[nodeID].Status) | ||||
|  | ||||
| 	// Wait till the stabilization period is over | ||||
| 	deadline := time.Now().Add(config.ServerStabilizationTime) | ||||
| 	healthy := false | ||||
| 	for time.Now().Before(deadline) { | ||||
| 		state, err := client.Sys().RaftAutopilotState() | ||||
| 		require.NoError(t, err) | ||||
| 		if state.Healthy { | ||||
| 			healthy = true | ||||
| 		} | ||||
| 		time.Sleep(1 * time.Second) | ||||
| 	} | ||||
| 	if !healthy { | ||||
| 		t.Fatalf("cluster failed to stabilize") | ||||
| 	} | ||||
| } | ||||
|  | ||||
| // joinAsVoterAndUnseal joins the specified core to the specified cluster as a voter and unseals it. | ||||
| // It will wait (up to a timeout) for the core to be fully unsealed before returning | ||||
| func joinAsVoterAndUnseal(t *testing.T, core *vault.TestClusterCore, cluster *vault.TestCluster) { | ||||
| 	joinAndUnseal(t, core, cluster, false, true) | ||||
| } | ||||
|  | ||||
| // joinAndUnseal joins the specified core to the specified cluster and unseals it. | ||||
| // You can specify if the core should be joined as a voter/non-voter, | ||||
| // and whether to wait (up to a timeout) for the core to be unsealed before returning. | ||||
| func joinAndUnseal(t *testing.T, core *vault.TestClusterCore, cluster *vault.TestCluster, nonVoter bool, waitForUnseal bool) { | ||||
| 	leader, leaderAddr := clusterLeader(t, cluster) | ||||
| 	_, err := core.JoinRaftCluster(namespace.RootContext(context.Background()), []*raft.LeaderJoinInfo{ | ||||
| 		{ | ||||
| 			LeaderAPIAddr: leaderAddr, | ||||
| 			TLSConfig:     leader.TLSConfig(), | ||||
| 			Retry:         true, | ||||
| 		}, | ||||
| 	}, nonVoter) | ||||
| 	require.NoError(t, err) | ||||
|  | ||||
| 	time.Sleep(1 * time.Second) | ||||
| 	cluster.UnsealCore(t, core) | ||||
| 	if waitForUnseal { | ||||
| 		waitForCoreUnseal(t, core) | ||||
| 	} | ||||
| } | ||||
|  | ||||
| // clusterLeader gets the leader node and its address from the specified cluster | ||||
| func clusterLeader(t *testing.T, cluster *vault.TestCluster) (*vault.TestClusterCore, string) { | ||||
| 	for _, core := range cluster.Cores { | ||||
| 		isLeader, addr, _, err := core.Leader() | ||||
| 		require.NoError(t, err) | ||||
| 		if isLeader { | ||||
| 			return core, addr | ||||
| 		} | ||||
| 	} | ||||
|  | ||||
| 	t.Fatal("unable to find leader") | ||||
| 	return nil, "" | ||||
| } | ||||
|  | ||||
| // setupLeaderAndUnseal configures and unseals the leader node. | ||||
| // It will wait until the node is active before returning the core and the address of the leader. | ||||
| func setupLeaderAndUnseal(t *testing.T, cluster *vault.TestCluster) (*vault.TestClusterCore, *testhelpers.TestRaftServerAddressProvider) { | ||||
| 	leader, _ := clusterLeader(t, cluster) | ||||
|  | ||||
| 	// Lots of tests seem to do this when they deal with a TestRaftServerAddressProvider, it makes the test work rather than error out. | ||||
| 	atomic.StoreUint32(&vault.TestingUpdateClusterAddr, 1) | ||||
|  | ||||
| 	addressProvider := &testhelpers.TestRaftServerAddressProvider{Cluster: cluster} | ||||
| 	testhelpers.EnsureCoreSealed(t, leader) | ||||
| 	leader.UnderlyingRawStorage.(*raft.RaftBackend).SetServerAddressProvider(addressProvider) | ||||
| 	cluster.UnsealCore(t, leader) | ||||
| 	vault.TestWaitActive(t, leader.Core) | ||||
|  | ||||
| 	return leader, addressProvider | ||||
| } | ||||
|  | ||||
| // waitForCoreUnseal waits until the specified core is unsealed. | ||||
| // It fails the calling test if the deadline has elapsed and the core is still sealed. | ||||
| func waitForCoreUnseal(t *testing.T, core *vault.TestClusterCore) { | ||||
| 	deadline := time.Now().Add(30 * time.Second) | ||||
| 	for time.Now().Before(deadline) { | ||||
| 		if !core.Sealed() { | ||||
| 			return | ||||
| 		} | ||||
| 		time.Sleep(time.Second) | ||||
| 	} | ||||
| 	t.Fatalf("expected core %v to unseal before deadline but it has not", core.NodeID) | ||||
| } | ||||
|  | ||||
| // isHealthyAfterStabilization will use the supplied leader core to query the | ||||
| // health of Raft Autopilot just after the specified deadline. | ||||
| func isHealthyAfterStabilization(t *testing.T, leaderCore *vault.TestClusterCore, stabilizationTime time.Duration) bool { | ||||
| 	timeoutGrace := 2 * time.Second | ||||
| 	time.Sleep(stabilizationTime + timeoutGrace) | ||||
| 	state, err := leaderCore.Client.Sys().RaftAutopilotState() | ||||
| 	require.NoError(t, err) | ||||
| 	require.NotNil(t, state) | ||||
| 	return state.Healthy | ||||
| } | ||||
|   | ||||
| @@ -1239,6 +1239,11 @@ func (c *Core) raftLeaderInfo(leaderInfo *raft.LeaderJoinInfo, disco *discover.D | ||||
| 	return ret, nil | ||||
| } | ||||
|  | ||||
| // NewDelegateForCore creates a raft.Delegate for the specified core using its backend. | ||||
| func NewDelegateForCore(c *Core) *raft.Delegate { | ||||
| 	return raft.NewDelegate(c.getRaftBackend()) | ||||
| } | ||||
|  | ||||
| // 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 { | ||||
|   | ||||
		Reference in New Issue
	
	Block a user
	 Peter Wilson
					Peter Wilson