mirror of https://github.com/status-im/consul.git
Switch to using the external autopilot module
This commit is contained in:
parent
7a2c6dfd62
commit
c048e86bb2
|
@ -1154,6 +1154,10 @@ func (b *Builder) Validate(rt RuntimeConfig) error {
|
|||
// check required params we cannot recover from first
|
||||
//
|
||||
|
||||
if rt.RaftProtocol != 3 {
|
||||
return fmt.Errorf("raft_protocol version %d is not supported by this version of Consul", rt.RaftProtocol)
|
||||
}
|
||||
|
||||
if err := validateBasicName("datacenter", rt.Datacenter, false); err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -119,6 +119,7 @@ func DefaultSource() Source {
|
|||
expose_min_port = 21500
|
||||
expose_max_port = 21755
|
||||
}
|
||||
raft_protocol = 3
|
||||
telemetry = {
|
||||
metrics_prefix = "consul"
|
||||
filter_default = true
|
||||
|
|
|
@ -662,14 +662,22 @@ func TestBuilder_BuildAndValidate_ConfigFlagsAndEdgecases(t *testing.T) {
|
|||
{
|
||||
desc: "-raft-protocol",
|
||||
args: []string{
|
||||
`-raft-protocol=1`,
|
||||
`-raft-protocol=3`,
|
||||
`-data-dir=` + dataDir,
|
||||
},
|
||||
patch: func(rt *RuntimeConfig) {
|
||||
rt.RaftProtocol = 1
|
||||
rt.RaftProtocol = 3
|
||||
rt.DataDir = dataDir
|
||||
},
|
||||
},
|
||||
{
|
||||
desc: "-raft-protocol unsupported",
|
||||
args: []string{
|
||||
`-raft-protocol=2`,
|
||||
`-data-dir=` + dataDir,
|
||||
},
|
||||
err: "raft_protocol version 2 is not supported by this version of Consul",
|
||||
},
|
||||
{
|
||||
desc: "-recursor",
|
||||
args: []string{
|
||||
|
@ -5302,7 +5310,7 @@ func TestFullConfig(t *testing.T) {
|
|||
"primary_datacenter": "ejtmd43d",
|
||||
"primary_gateways": [ "aej8eeZo", "roh2KahS" ],
|
||||
"primary_gateways_interval": "18866s",
|
||||
"raft_protocol": 19016,
|
||||
"raft_protocol": 3,
|
||||
"raft_snapshot_threshold": 16384,
|
||||
"raft_snapshot_interval": "30s",
|
||||
"raft_trailing_logs": 83749,
|
||||
|
@ -5991,7 +5999,7 @@ func TestFullConfig(t *testing.T) {
|
|||
primary_datacenter = "ejtmd43d"
|
||||
primary_gateways = [ "aej8eeZo", "roh2KahS" ]
|
||||
primary_gateways_interval = "18866s"
|
||||
raft_protocol = 19016
|
||||
raft_protocol = 3
|
||||
raft_snapshot_threshold = 16384
|
||||
raft_snapshot_interval = "30s"
|
||||
raft_trailing_logs = 83749
|
||||
|
@ -6753,7 +6761,7 @@ func TestFullConfig(t *testing.T) {
|
|||
RPCRateLimit: 12029.43,
|
||||
RPCMaxBurst: 44848,
|
||||
RPCMaxConnsPerClient: 2954,
|
||||
RaftProtocol: 19016,
|
||||
RaftProtocol: 3,
|
||||
RaftSnapshotThreshold: 16384,
|
||||
RaftSnapshotInterval: 30 * time.Second,
|
||||
RaftTrailingLogs: 83749,
|
||||
|
@ -7434,6 +7442,7 @@ func TestSanitize(t *testing.T) {
|
|||
EntryFetchRate: 0.334,
|
||||
},
|
||||
ConsulCoordinateUpdatePeriod: 15 * time.Second,
|
||||
RaftProtocol: 3,
|
||||
RetryJoinLAN: []string{
|
||||
"foo=bar key=baz secret=boom bang=bar",
|
||||
},
|
||||
|
@ -7690,7 +7699,7 @@ func TestSanitize(t *testing.T) {
|
|||
"RPCConfig": {
|
||||
"EnableStreaming": false
|
||||
},
|
||||
"RaftProtocol": 0,
|
||||
"RaftProtocol": 3,
|
||||
"RaftSnapshotInterval": "0s",
|
||||
"RaftSnapshotThreshold": 0,
|
||||
"RaftTrailingLogs": 0,
|
||||
|
|
|
@ -3,13 +3,12 @@ package consul
|
|||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
"strconv"
|
||||
|
||||
"github.com/armon/go-metrics"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/metadata"
|
||||
"github.com/hashicorp/consul/types"
|
||||
"github.com/hashicorp/raft"
|
||||
autopilot "github.com/hashicorp/raft-autopilot"
|
||||
"github.com/hashicorp/serf/serf"
|
||||
)
|
||||
|
||||
|
@ -19,44 +18,22 @@ type AutopilotDelegate struct {
|
|||
}
|
||||
|
||||
func (d *AutopilotDelegate) AutopilotConfig() *autopilot.Config {
|
||||
return d.server.getOrCreateAutopilotConfig()
|
||||
return d.server.getOrCreateAutopilotConfig().ToAutopilotLibraryConfig()
|
||||
}
|
||||
|
||||
func (d *AutopilotDelegate) FetchStats(ctx context.Context, servers []serf.Member) map[string]*autopilot.ServerStats {
|
||||
func (d *AutopilotDelegate) KnownServers() map[raft.ServerID]*autopilot.Server {
|
||||
return d.server.autopilotServers()
|
||||
}
|
||||
|
||||
func (d *AutopilotDelegate) FetchServerStats(ctx context.Context, servers map[raft.ServerID]*autopilot.Server) map[raft.ServerID]*autopilot.ServerStats {
|
||||
return d.server.statsFetcher.Fetch(ctx, servers)
|
||||
}
|
||||
|
||||
func (d *AutopilotDelegate) IsServer(m serf.Member) (*autopilot.ServerInfo, error) {
|
||||
if m.Tags["role"] != "consul" {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
portStr := m.Tags["port"]
|
||||
port, err := strconv.Atoi(portStr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
buildVersion, err := metadata.Build(&m)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
server := &autopilot.ServerInfo{
|
||||
Name: m.Name,
|
||||
ID: m.Tags["id"],
|
||||
Addr: &net.TCPAddr{IP: m.Addr, Port: port},
|
||||
Build: *buildVersion,
|
||||
Status: m.Status,
|
||||
}
|
||||
return server, nil
|
||||
}
|
||||
|
||||
// Heartbeat a metric for monitoring if we're the leader
|
||||
func (d *AutopilotDelegate) NotifyHealth(health autopilot.OperatorHealthReply) {
|
||||
func (d *AutopilotDelegate) NotifyState(state *autopilot.State) {
|
||||
// emit metrics if we are the leader regarding overall healthiness and the failure tolerance
|
||||
if d.server.raft.State() == raft.Leader {
|
||||
metrics.SetGauge([]string{"autopilot", "failure_tolerance"}, float32(health.FailureTolerance))
|
||||
if health.Healthy {
|
||||
metrics.SetGauge([]string{"autopilot", "failure_tolerance"}, float32(state.FailureTolerance))
|
||||
if state.Healthy {
|
||||
metrics.SetGauge([]string{"autopilot", "healthy"}, 1)
|
||||
} else {
|
||||
metrics.SetGauge([]string{"autopilot", "healthy"}, 0)
|
||||
|
@ -64,23 +41,88 @@ func (d *AutopilotDelegate) NotifyHealth(health autopilot.OperatorHealthReply) {
|
|||
}
|
||||
}
|
||||
|
||||
func (d *AutopilotDelegate) PromoteNonVoters(conf *autopilot.Config, health autopilot.OperatorHealthReply) ([]raft.Server, error) {
|
||||
future := d.server.raft.GetConfiguration()
|
||||
if err := future.Error(); err != nil {
|
||||
return nil, fmt.Errorf("failed to get raft configuration: %v", err)
|
||||
func (d *AutopilotDelegate) RemoveFailedServer(srv *autopilot.Server) error {
|
||||
if err := d.server.RemoveFailedNode(srv.Name, false); err != nil {
|
||||
return fmt.Errorf("failed to remove server: %w", err)
|
||||
}
|
||||
|
||||
return autopilot.PromoteStableServers(conf, health, future.Configuration().Servers), nil
|
||||
return nil
|
||||
}
|
||||
|
||||
func (d *AutopilotDelegate) Raft() *raft.Raft {
|
||||
return d.server.raft
|
||||
func (s *Server) initAutopilot(config *Config) {
|
||||
apDelegate := &AutopilotDelegate{s}
|
||||
|
||||
s.autopilot = autopilot.New(
|
||||
s.raft,
|
||||
apDelegate,
|
||||
autopilot.WithLogger(s.logger),
|
||||
autopilot.WithReconcileInterval(config.AutopilotInterval),
|
||||
autopilot.WithUpdateInterval(config.ServerHealthInterval),
|
||||
autopilot.WithPromoter(s.autopilotPromoter()),
|
||||
)
|
||||
}
|
||||
|
||||
func (d *AutopilotDelegate) SerfLAN() *serf.Serf {
|
||||
return d.server.serfLAN
|
||||
func (s *Server) autopilotServers() map[raft.ServerID]*autopilot.Server {
|
||||
servers := make(map[raft.ServerID]*autopilot.Server)
|
||||
for _, member := range s.serfLAN.Members() {
|
||||
srv, err := s.autopilotServer(member)
|
||||
if err != nil {
|
||||
s.logger.Warn("Error parsing server info", "name", member.Name, "error", err)
|
||||
continue
|
||||
} else if srv == nil {
|
||||
// this member was a client
|
||||
continue
|
||||
}
|
||||
|
||||
servers[srv.ID] = srv
|
||||
}
|
||||
|
||||
return servers
|
||||
}
|
||||
|
||||
func (d *AutopilotDelegate) SerfWAN() *serf.Serf {
|
||||
return d.server.serfWAN
|
||||
func (s *Server) autopilotServer(m serf.Member) (*autopilot.Server, error) {
|
||||
ok, srv := metadata.IsConsulServer(m)
|
||||
if !ok {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
return s.autopilotServerFromMetadata(srv)
|
||||
}
|
||||
|
||||
func (s *Server) autopilotServerFromMetadata(srv *metadata.Server) (*autopilot.Server, error) {
|
||||
server := &autopilot.Server{
|
||||
Name: srv.ShortName,
|
||||
ID: raft.ServerID(srv.ID),
|
||||
Address: raft.ServerAddress(srv.Addr.String()),
|
||||
Version: srv.Build.String(),
|
||||
RaftVersion: srv.RaftVersion,
|
||||
Ext: s.autopilotServerExt(srv),
|
||||
}
|
||||
|
||||
switch srv.Status {
|
||||
case serf.StatusLeft:
|
||||
server.NodeStatus = autopilot.NodeLeft
|
||||
case serf.StatusAlive, serf.StatusLeaving:
|
||||
// we want to treat leaving as alive to prevent autopilot from
|
||||
// prematurely removing the node.
|
||||
server.NodeStatus = autopilot.NodeAlive
|
||||
case serf.StatusFailed:
|
||||
server.NodeStatus = autopilot.NodeFailed
|
||||
default:
|
||||
server.NodeStatus = autopilot.NodeUnknown
|
||||
}
|
||||
|
||||
// populate the node meta if there is any. When a node first joins or if
|
||||
// there are ACL issues then this could be empty if the server has not
|
||||
// yet been able to register itself in the catalog
|
||||
_, node, err := s.fsm.State().GetNodeID(types.NodeID(srv.ID))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("error retrieving node from state store: %w", err)
|
||||
}
|
||||
|
||||
if node != nil {
|
||||
server.Meta = node.Meta
|
||||
}
|
||||
|
||||
return server, nil
|
||||
}
|
||||
|
|
|
@ -1,540 +0,0 @@
|
|||
package autopilot
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/logging"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/go-version"
|
||||
"github.com/hashicorp/raft"
|
||||
"github.com/hashicorp/serf/serf"
|
||||
)
|
||||
|
||||
// Delegate is the interface for the Autopilot mechanism
|
||||
type Delegate interface {
|
||||
AutopilotConfig() *Config
|
||||
FetchStats(context.Context, []serf.Member) map[string]*ServerStats
|
||||
IsServer(serf.Member) (*ServerInfo, error)
|
||||
NotifyHealth(OperatorHealthReply)
|
||||
PromoteNonVoters(*Config, OperatorHealthReply) ([]raft.Server, error)
|
||||
Raft() *raft.Raft
|
||||
SerfLAN() *serf.Serf
|
||||
SerfWAN() *serf.Serf
|
||||
}
|
||||
|
||||
// Autopilot is a mechanism for automatically managing the Raft
|
||||
// quorum using server health information along with updates from Serf gossip.
|
||||
// For more information, see https://www.consul.io/docs/guides/autopilot.html
|
||||
type Autopilot struct {
|
||||
logger hclog.Logger
|
||||
delegate Delegate
|
||||
|
||||
interval time.Duration
|
||||
healthInterval time.Duration
|
||||
|
||||
clusterHealth OperatorHealthReply
|
||||
clusterHealthLock sync.RWMutex
|
||||
|
||||
enabled bool
|
||||
removeDeadCh chan struct{}
|
||||
shutdownCh chan struct{}
|
||||
shutdownLock sync.Mutex
|
||||
waitGroup sync.WaitGroup
|
||||
}
|
||||
|
||||
type ServerInfo struct {
|
||||
Name string
|
||||
ID string
|
||||
Addr net.Addr
|
||||
Build version.Version
|
||||
Status serf.MemberStatus
|
||||
}
|
||||
|
||||
func NewAutopilot(logger hclog.Logger, delegate Delegate, interval, healthInterval time.Duration) *Autopilot {
|
||||
return &Autopilot{
|
||||
logger: logger.Named(logging.Autopilot),
|
||||
delegate: delegate,
|
||||
interval: interval,
|
||||
healthInterval: healthInterval,
|
||||
removeDeadCh: make(chan struct{}),
|
||||
}
|
||||
}
|
||||
|
||||
func (a *Autopilot) Start() {
|
||||
a.shutdownLock.Lock()
|
||||
defer a.shutdownLock.Unlock()
|
||||
|
||||
// Nothing to do
|
||||
if a.enabled {
|
||||
return
|
||||
}
|
||||
|
||||
a.shutdownCh = make(chan struct{})
|
||||
a.waitGroup = sync.WaitGroup{}
|
||||
a.clusterHealth = OperatorHealthReply{}
|
||||
|
||||
a.waitGroup.Add(2)
|
||||
go a.run()
|
||||
go a.serverHealthLoop()
|
||||
a.enabled = true
|
||||
}
|
||||
|
||||
func (a *Autopilot) Stop() {
|
||||
a.shutdownLock.Lock()
|
||||
defer a.shutdownLock.Unlock()
|
||||
|
||||
// Nothing to do
|
||||
if !a.enabled {
|
||||
return
|
||||
}
|
||||
|
||||
close(a.shutdownCh)
|
||||
a.waitGroup.Wait()
|
||||
a.enabled = false
|
||||
}
|
||||
|
||||
// run periodically looks for nonvoting servers to promote and dead servers to remove.
|
||||
func (a *Autopilot) run() {
|
||||
defer a.waitGroup.Done()
|
||||
|
||||
// Monitor server health until shutdown
|
||||
ticker := time.NewTicker(a.interval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-a.shutdownCh:
|
||||
return
|
||||
case <-ticker.C:
|
||||
if err := a.promoteServers(); err != nil {
|
||||
a.logger.Error("Error promoting servers", "error", err)
|
||||
}
|
||||
|
||||
if err := a.pruneDeadServers(); err != nil {
|
||||
a.logger.Error("Error checking for dead servers to remove", "error", err)
|
||||
}
|
||||
case <-a.removeDeadCh:
|
||||
if err := a.pruneDeadServers(); err != nil {
|
||||
a.logger.Error("Error checking for dead servers to remove", "error", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// promoteServers asks the delegate for any promotions and carries them out.
|
||||
func (a *Autopilot) promoteServers() error {
|
||||
conf := a.delegate.AutopilotConfig()
|
||||
if conf == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Skip the non-voter promotions unless all servers support the new APIs
|
||||
minRaftProtocol, err := a.MinRaftProtocol()
|
||||
if err != nil {
|
||||
return fmt.Errorf("error getting server raft protocol versions: %s", err)
|
||||
}
|
||||
if minRaftProtocol >= 3 {
|
||||
promotions, err := a.delegate.PromoteNonVoters(conf, a.GetClusterHealth())
|
||||
if err != nil {
|
||||
return fmt.Errorf("error checking for non-voters to promote: %s", err)
|
||||
}
|
||||
if err := a.handlePromotions(promotions); err != nil {
|
||||
return fmt.Errorf("error handling promotions: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// fmtServer prints info about a server in a standard way for logging.
|
||||
func fmtServer(server raft.Server) string {
|
||||
return fmt.Sprintf("Server (ID: %q Address: %q)", server.ID, server.Address)
|
||||
}
|
||||
|
||||
// NumPeers counts the number of voting peers in the given raft config.
|
||||
func NumPeers(raftConfig raft.Configuration) int {
|
||||
var numPeers int
|
||||
for _, server := range raftConfig.Servers {
|
||||
if server.Suffrage == raft.Voter {
|
||||
numPeers++
|
||||
}
|
||||
}
|
||||
return numPeers
|
||||
}
|
||||
|
||||
// RemoveDeadServers triggers a pruning of dead servers in a non-blocking way.
|
||||
func (a *Autopilot) RemoveDeadServers() {
|
||||
select {
|
||||
case a.removeDeadCh <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
func canRemoveServers(peers, minQuorum, deadServers int) (bool, string) {
|
||||
if peers-deadServers < minQuorum {
|
||||
return false, fmt.Sprintf("denied, because removing %d/%d servers would leave less then minimal allowed quorum of %d servers", deadServers, peers, minQuorum)
|
||||
}
|
||||
|
||||
// Only do removals if a minority of servers will be affected.
|
||||
// For failure tolerance of F we need n = 2F+1 servers.
|
||||
// This means we can safely remove up to (n-1)/2 servers.
|
||||
if deadServers > (peers-1)/2 {
|
||||
return false, fmt.Sprintf("denied, because removing the majority of servers %d/%d is not safe", deadServers, peers)
|
||||
}
|
||||
return true, fmt.Sprintf("allowed, because removing %d/%d servers leaves a majority of servers above the minimal allowed quorum %d", deadServers, peers, minQuorum)
|
||||
}
|
||||
|
||||
// pruneDeadServers removes up to numPeers/2 failed servers
|
||||
func (a *Autopilot) pruneDeadServers() error {
|
||||
conf := a.delegate.AutopilotConfig()
|
||||
if conf == nil || !conf.CleanupDeadServers {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Failed servers are known to Serf and marked failed, and stale servers
|
||||
// are known to Raft but not Serf.
|
||||
var failed []serf.Member
|
||||
staleRaftServers := make(map[string]raft.Server)
|
||||
raftNode := a.delegate.Raft()
|
||||
future := raftNode.GetConfiguration()
|
||||
if err := future.Error(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
raftConfig := future.Configuration()
|
||||
for _, server := range raftConfig.Servers {
|
||||
staleRaftServers[string(server.Address)] = server
|
||||
}
|
||||
serfWAN := a.delegate.SerfWAN()
|
||||
serfLAN := a.delegate.SerfLAN()
|
||||
for _, member := range serfLAN.Members() {
|
||||
server, err := a.delegate.IsServer(member)
|
||||
if err != nil {
|
||||
a.logger.Warn("Error parsing server info", "name", member.Name, "error", err)
|
||||
continue
|
||||
}
|
||||
if server != nil {
|
||||
// todo(kyhavlov): change this to index by UUID
|
||||
s, found := staleRaftServers[server.Addr.String()]
|
||||
if found {
|
||||
delete(staleRaftServers, server.Addr.String())
|
||||
}
|
||||
|
||||
if member.Status == serf.StatusFailed {
|
||||
// If the node is a nonvoter, we can remove it immediately.
|
||||
if found && s.Suffrage == raft.Nonvoter {
|
||||
a.logger.Info("Attempting removal of failed server node", "name", member.Name)
|
||||
go serfLAN.RemoveFailedNode(member.Name)
|
||||
if serfWAN != nil {
|
||||
go serfWAN.RemoveFailedNode(member.Name)
|
||||
}
|
||||
} else {
|
||||
failed = append(failed, member)
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
deadServers := len(failed) + len(staleRaftServers)
|
||||
|
||||
// nothing to do
|
||||
if deadServers == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
if ok, msg := canRemoveServers(NumPeers(raftConfig), int(conf.MinQuorum), deadServers); !ok {
|
||||
a.logger.Debug("Failed to remove dead servers", "error", msg)
|
||||
return nil
|
||||
}
|
||||
|
||||
for _, node := range failed {
|
||||
a.logger.Info("Attempting removal of failed server node", "name", node.Name)
|
||||
go serfLAN.RemoveFailedNode(node.Name)
|
||||
if serfWAN != nil {
|
||||
go serfWAN.RemoveFailedNode(fmt.Sprintf("%s.%s", node.Name, node.Tags["dc"]))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
minRaftProtocol, err := a.MinRaftProtocol()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, raftServer := range staleRaftServers {
|
||||
a.logger.Info("Attempting removal of stale server", "server", fmtServer(raftServer))
|
||||
var future raft.Future
|
||||
if minRaftProtocol >= 2 {
|
||||
future = raftNode.RemoveServer(raftServer.ID, 0, 0)
|
||||
} else {
|
||||
future = raftNode.RemovePeer(raftServer.Address)
|
||||
}
|
||||
if err := future.Error(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// MinRaftProtocol returns the lowest supported Raft protocol among alive servers
|
||||
func (a *Autopilot) MinRaftProtocol() (int, error) {
|
||||
return minRaftProtocol(a.delegate.SerfLAN().Members(), a.delegate.IsServer)
|
||||
}
|
||||
|
||||
func minRaftProtocol(members []serf.Member, serverFunc func(serf.Member) (*ServerInfo, error)) (int, error) {
|
||||
minVersion := -1
|
||||
for _, m := range members {
|
||||
if m.Status != serf.StatusAlive {
|
||||
continue
|
||||
}
|
||||
|
||||
server, err := serverFunc(m)
|
||||
if err != nil {
|
||||
return -1, err
|
||||
}
|
||||
if server == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
vsn, ok := m.Tags["raft_vsn"]
|
||||
if !ok {
|
||||
vsn = "1"
|
||||
}
|
||||
raftVsn, err := strconv.Atoi(vsn)
|
||||
if err != nil {
|
||||
return -1, err
|
||||
}
|
||||
|
||||
if minVersion == -1 || raftVsn < minVersion {
|
||||
minVersion = raftVsn
|
||||
}
|
||||
}
|
||||
|
||||
if minVersion == -1 {
|
||||
return minVersion, fmt.Errorf("No servers found")
|
||||
}
|
||||
|
||||
return minVersion, nil
|
||||
}
|
||||
|
||||
// handlePromotions is a helper shared with Consul Enterprise that attempts to
|
||||
// apply desired server promotions to the Raft configuration.
|
||||
func (a *Autopilot) handlePromotions(promotions []raft.Server) error {
|
||||
// This used to wait to only promote to maintain an odd quorum of
|
||||
// servers, but this was at odds with the dead server cleanup when doing
|
||||
// rolling updates (add one new server, wait, and then kill an old
|
||||
// server). The dead server cleanup would still count the old server as
|
||||
// a peer, which is conservative and the right thing to do, and this
|
||||
// would wait to promote, so you could get into a stalemate. It is safer
|
||||
// to promote early than remove early, so by promoting as soon as
|
||||
// possible we have chosen that as the solution here.
|
||||
for _, server := range promotions {
|
||||
a.logger.Info("Promoting server to voter", "server", fmtServer(server))
|
||||
addFuture := a.delegate.Raft().AddVoter(server.ID, server.Address, 0, 0)
|
||||
if err := addFuture.Error(); err != nil {
|
||||
return fmt.Errorf("failed to add raft peer: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// If we promoted a server, trigger a check to remove dead servers.
|
||||
if len(promotions) > 0 {
|
||||
select {
|
||||
case a.removeDeadCh <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// serverHealthLoop monitors the health of the servers in the cluster
|
||||
func (a *Autopilot) serverHealthLoop() {
|
||||
defer a.waitGroup.Done()
|
||||
|
||||
// Monitor server health until shutdown
|
||||
ticker := time.NewTicker(a.healthInterval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-a.shutdownCh:
|
||||
return
|
||||
case <-ticker.C:
|
||||
if err := a.updateClusterHealth(); err != nil {
|
||||
a.logger.Error("Error updating cluster health", "error", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// updateClusterHealth fetches the Raft stats of the other servers and updates
|
||||
// s.clusterHealth based on the configured Autopilot thresholds
|
||||
func (a *Autopilot) updateClusterHealth() error {
|
||||
// Don't do anything if the min Raft version is too low
|
||||
minRaftProtocol, err := a.MinRaftProtocol()
|
||||
if err != nil {
|
||||
return fmt.Errorf("error getting server raft protocol versions: %s", err)
|
||||
}
|
||||
if minRaftProtocol < 3 {
|
||||
return nil
|
||||
}
|
||||
|
||||
autopilotConf := a.delegate.AutopilotConfig()
|
||||
// Bail early if autopilot config hasn't been initialized yet
|
||||
if autopilotConf == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Get the the serf members which are Consul servers
|
||||
var serverMembers []serf.Member
|
||||
serverMap := make(map[string]*ServerInfo)
|
||||
for _, member := range a.delegate.SerfLAN().Members() {
|
||||
if member.Status == serf.StatusLeft {
|
||||
continue
|
||||
}
|
||||
|
||||
server, err := a.delegate.IsServer(member)
|
||||
if err != nil {
|
||||
a.logger.Warn("Error parsing server info", "name", member.Name, "error", err)
|
||||
continue
|
||||
}
|
||||
if server != nil {
|
||||
serverMap[server.ID] = server
|
||||
serverMembers = append(serverMembers, member)
|
||||
}
|
||||
}
|
||||
|
||||
raftNode := a.delegate.Raft()
|
||||
future := raftNode.GetConfiguration()
|
||||
if err := future.Error(); err != nil {
|
||||
return fmt.Errorf("error getting Raft configuration %s", err)
|
||||
}
|
||||
servers := future.Configuration().Servers
|
||||
|
||||
// Fetch the health for each of the servers in parallel so we get as
|
||||
// consistent of a sample as possible. We capture the leader's index
|
||||
// here as well so it roughly lines up with the same point in time.
|
||||
targetLastIndex := raftNode.LastIndex()
|
||||
d := time.Now().Add(a.healthInterval / 2)
|
||||
ctx, cancel := context.WithDeadline(context.Background(), d)
|
||||
defer cancel()
|
||||
fetchedStats := a.delegate.FetchStats(ctx, serverMembers)
|
||||
|
||||
// Build a current list of server healths
|
||||
leader := raftNode.Leader()
|
||||
var clusterHealth OperatorHealthReply
|
||||
voterCount := 0
|
||||
healthyCount := 0
|
||||
healthyVoterCount := 0
|
||||
for _, server := range servers {
|
||||
health := ServerHealth{
|
||||
ID: string(server.ID),
|
||||
Address: string(server.Address),
|
||||
Leader: server.Address == leader,
|
||||
LastContact: -1,
|
||||
Voter: server.Suffrage == raft.Voter,
|
||||
}
|
||||
|
||||
parts, ok := serverMap[string(server.ID)]
|
||||
if ok {
|
||||
health.Name = parts.Name
|
||||
health.SerfStatus = parts.Status
|
||||
health.Version = parts.Build.String()
|
||||
if stats, ok := fetchedStats[string(server.ID)]; ok {
|
||||
if err := a.updateServerHealth(&health, parts, stats, autopilotConf, targetLastIndex); err != nil {
|
||||
a.logger.Warn("Error updating server health", "server", fmtServer(server), "error", err)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
health.SerfStatus = serf.StatusNone
|
||||
}
|
||||
|
||||
if health.Voter {
|
||||
voterCount++
|
||||
}
|
||||
if health.Healthy {
|
||||
healthyCount++
|
||||
if health.Voter {
|
||||
healthyVoterCount++
|
||||
}
|
||||
}
|
||||
|
||||
clusterHealth.Servers = append(clusterHealth.Servers, health)
|
||||
}
|
||||
clusterHealth.Healthy = healthyCount == len(servers)
|
||||
|
||||
// If we have extra healthy voters, update FailureTolerance
|
||||
requiredQuorum := voterCount/2 + 1
|
||||
if healthyVoterCount > requiredQuorum {
|
||||
clusterHealth.FailureTolerance = healthyVoterCount - requiredQuorum
|
||||
}
|
||||
|
||||
a.delegate.NotifyHealth(clusterHealth)
|
||||
|
||||
a.clusterHealthLock.Lock()
|
||||
a.clusterHealth = clusterHealth
|
||||
a.clusterHealthLock.Unlock()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// updateServerHealth computes the resulting health of the server based on its
|
||||
// fetched stats and the state of the leader.
|
||||
func (a *Autopilot) updateServerHealth(health *ServerHealth,
|
||||
server *ServerInfo, stats *ServerStats,
|
||||
autopilotConf *Config, targetLastIndex uint64) error {
|
||||
|
||||
health.LastTerm = stats.LastTerm
|
||||
health.LastIndex = stats.LastIndex
|
||||
|
||||
if stats.LastContact != "never" {
|
||||
var err error
|
||||
health.LastContact, err = time.ParseDuration(stats.LastContact)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error parsing last_contact duration: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
raftNode := a.delegate.Raft()
|
||||
lastTerm, err := strconv.ParseUint(raftNode.Stats()["last_log_term"], 10, 64)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error parsing last_log_term: %s", err)
|
||||
}
|
||||
health.Healthy = health.IsHealthy(lastTerm, targetLastIndex, autopilotConf)
|
||||
|
||||
// If this is a new server or the health changed, reset StableSince
|
||||
lastHealth := a.GetServerHealth(server.ID)
|
||||
if lastHealth == nil || lastHealth.Healthy != health.Healthy {
|
||||
health.StableSince = time.Now()
|
||||
} else {
|
||||
health.StableSince = lastHealth.StableSince
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (a *Autopilot) GetClusterHealth() OperatorHealthReply {
|
||||
a.clusterHealthLock.RLock()
|
||||
defer a.clusterHealthLock.RUnlock()
|
||||
return a.clusterHealth
|
||||
}
|
||||
|
||||
func (a *Autopilot) GetServerHealth(id string) *ServerHealth {
|
||||
a.clusterHealthLock.RLock()
|
||||
defer a.clusterHealthLock.RUnlock()
|
||||
return a.clusterHealth.ServerHealth(id)
|
||||
}
|
||||
|
||||
func IsPotentialVoter(suffrage raft.ServerSuffrage) bool {
|
||||
switch suffrage {
|
||||
case raft.Voter, raft.Staging:
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
|
@ -1,111 +0,0 @@
|
|||
package autopilot
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"net"
|
||||
"testing"
|
||||
|
||||
"github.com/hashicorp/serf/serf"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestMinRaftProtocol(t *testing.T) {
|
||||
t.Parallel()
|
||||
makeMember := func(version string) serf.Member {
|
||||
return serf.Member{
|
||||
Name: "foo",
|
||||
Addr: net.IP([]byte{127, 0, 0, 1}),
|
||||
Tags: map[string]string{
|
||||
"role": "consul",
|
||||
"dc": "dc1",
|
||||
"port": "10000",
|
||||
"vsn": "1",
|
||||
"raft_vsn": version,
|
||||
},
|
||||
Status: serf.StatusAlive,
|
||||
}
|
||||
}
|
||||
|
||||
cases := []struct {
|
||||
members []serf.Member
|
||||
expected int
|
||||
err error
|
||||
}{
|
||||
// No servers, error
|
||||
{
|
||||
members: []serf.Member{},
|
||||
expected: -1,
|
||||
err: errors.New("No servers found"),
|
||||
},
|
||||
// One server
|
||||
{
|
||||
members: []serf.Member{
|
||||
makeMember("1"),
|
||||
},
|
||||
expected: 1,
|
||||
},
|
||||
// One server, bad version formatting
|
||||
{
|
||||
members: []serf.Member{
|
||||
makeMember("asdf"),
|
||||
},
|
||||
expected: -1,
|
||||
err: errors.New(`strconv.Atoi: parsing "asdf": invalid syntax`),
|
||||
},
|
||||
// Multiple servers, different versions
|
||||
{
|
||||
members: []serf.Member{
|
||||
makeMember("1"),
|
||||
makeMember("2"),
|
||||
},
|
||||
expected: 1,
|
||||
},
|
||||
// Multiple servers, same version
|
||||
{
|
||||
members: []serf.Member{
|
||||
makeMember("2"),
|
||||
makeMember("2"),
|
||||
},
|
||||
expected: 2,
|
||||
},
|
||||
}
|
||||
|
||||
serverFunc := func(m serf.Member) (*ServerInfo, error) {
|
||||
return &ServerInfo{}, nil
|
||||
}
|
||||
for _, tc := range cases {
|
||||
result, err := minRaftProtocol(tc.members, serverFunc)
|
||||
if result != tc.expected {
|
||||
t.Fatalf("bad: %v, %v, %v", result, tc.expected, tc)
|
||||
}
|
||||
if tc.err != nil {
|
||||
if err == nil || tc.err.Error() != err.Error() {
|
||||
t.Fatalf("bad: %v, %v, %v", err, tc.err, tc)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestAutopilot_canRemoveServers(t *testing.T) {
|
||||
type test struct {
|
||||
peers int
|
||||
minQuorum int
|
||||
deadServers int
|
||||
ok bool
|
||||
}
|
||||
|
||||
tests := []test{
|
||||
{1, 1, 1, false},
|
||||
{3, 3, 1, false},
|
||||
{4, 3, 3, false},
|
||||
{5, 3, 3, false},
|
||||
{5, 3, 2, true},
|
||||
{5, 3, 1, true},
|
||||
{9, 3, 5, false},
|
||||
}
|
||||
for _, test := range tests {
|
||||
ok, msg := canRemoveServers(test.peers, test.minQuorum, test.deadServers)
|
||||
require.Equal(t, test.ok, ok)
|
||||
t.Logf("%+v: %s", test, msg)
|
||||
}
|
||||
}
|
|
@ -1,26 +0,0 @@
|
|||
package autopilot
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/raft"
|
||||
)
|
||||
|
||||
// PromoteStableServers is a basic autopilot promotion policy that promotes any
|
||||
// server which has been healthy and stable for the duration specified in the
|
||||
// given Autopilot config.
|
||||
func PromoteStableServers(autopilotConfig *Config, health OperatorHealthReply, servers []raft.Server) []raft.Server {
|
||||
// Find any non-voters eligible for promotion.
|
||||
now := time.Now()
|
||||
var promotions []raft.Server
|
||||
for _, server := range servers {
|
||||
if !IsPotentialVoter(server.Suffrage) {
|
||||
health := health.ServerHealth(string(server.ID))
|
||||
if health.IsStable(now, autopilotConfig) {
|
||||
promotions = append(promotions, server)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return promotions
|
||||
}
|
|
@ -1,102 +0,0 @@
|
|||
package autopilot
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/raft"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestPromotion(t *testing.T) {
|
||||
config := &Config{
|
||||
LastContactThreshold: 5 * time.Second,
|
||||
MaxTrailingLogs: 100,
|
||||
ServerStabilizationTime: 3 * time.Second,
|
||||
}
|
||||
|
||||
cases := []struct {
|
||||
name string
|
||||
conf *Config
|
||||
health OperatorHealthReply
|
||||
servers []raft.Server
|
||||
promotions []raft.Server
|
||||
}{
|
||||
{
|
||||
name: "one stable voter, no promotions",
|
||||
conf: config,
|
||||
health: OperatorHealthReply{
|
||||
Servers: []ServerHealth{
|
||||
{
|
||||
ID: "a",
|
||||
Healthy: true,
|
||||
StableSince: time.Now().Add(-10 * time.Second),
|
||||
},
|
||||
},
|
||||
},
|
||||
servers: []raft.Server{
|
||||
{ID: "a", Suffrage: raft.Voter},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "one stable nonvoter, should be promoted",
|
||||
conf: config,
|
||||
health: OperatorHealthReply{
|
||||
Servers: []ServerHealth{
|
||||
{
|
||||
ID: "a",
|
||||
Healthy: true,
|
||||
StableSince: time.Now().Add(-10 * time.Second),
|
||||
},
|
||||
{
|
||||
ID: "b",
|
||||
Healthy: true,
|
||||
StableSince: time.Now().Add(-10 * time.Second),
|
||||
},
|
||||
},
|
||||
},
|
||||
servers: []raft.Server{
|
||||
{ID: "a", Suffrage: raft.Voter},
|
||||
{ID: "b", Suffrage: raft.Nonvoter},
|
||||
},
|
||||
promotions: []raft.Server{
|
||||
{ID: "b", Suffrage: raft.Nonvoter},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "unstable servers, neither should be promoted",
|
||||
conf: config,
|
||||
health: OperatorHealthReply{
|
||||
Servers: []ServerHealth{
|
||||
{
|
||||
ID: "a",
|
||||
Healthy: true,
|
||||
StableSince: time.Now().Add(-10 * time.Second),
|
||||
},
|
||||
{
|
||||
ID: "b",
|
||||
Healthy: false,
|
||||
StableSince: time.Now().Add(-10 * time.Second),
|
||||
},
|
||||
{
|
||||
ID: "c",
|
||||
Healthy: true,
|
||||
StableSince: time.Now().Add(-1 * time.Second),
|
||||
},
|
||||
},
|
||||
},
|
||||
servers: []raft.Server{
|
||||
{ID: "a", Suffrage: raft.Voter},
|
||||
{ID: "b", Suffrage: raft.Nonvoter},
|
||||
{ID: "c", Suffrage: raft.Nonvoter},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range cases {
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
promotions := PromoteStableServers(tc.conf, tc.health, tc.servers)
|
||||
require.Equal(t, tc.promotions, promotions)
|
||||
})
|
||||
}
|
||||
}
|
|
@ -1,94 +0,0 @@
|
|||
package autopilot
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/serf/serf"
|
||||
)
|
||||
|
||||
func TestServerHealth_IsHealthy(t *testing.T) {
|
||||
cases := []struct {
|
||||
health ServerHealth
|
||||
lastTerm uint64
|
||||
lastIndex uint64
|
||||
conf Config
|
||||
expected bool
|
||||
}{
|
||||
// Healthy server, all values within allowed limits
|
||||
{
|
||||
health: ServerHealth{SerfStatus: serf.StatusAlive, LastTerm: 1, LastIndex: 0},
|
||||
lastTerm: 1,
|
||||
lastIndex: 10,
|
||||
conf: Config{MaxTrailingLogs: 20},
|
||||
expected: true,
|
||||
},
|
||||
// Serf status failed
|
||||
{
|
||||
health: ServerHealth{SerfStatus: serf.StatusFailed},
|
||||
expected: false,
|
||||
},
|
||||
// Old value for lastTerm
|
||||
{
|
||||
health: ServerHealth{SerfStatus: serf.StatusAlive, LastTerm: 0},
|
||||
lastTerm: 1,
|
||||
expected: false,
|
||||
},
|
||||
// Too far behind on logs
|
||||
{
|
||||
health: ServerHealth{SerfStatus: serf.StatusAlive, LastIndex: 0},
|
||||
lastIndex: 10,
|
||||
conf: Config{MaxTrailingLogs: 5},
|
||||
expected: false,
|
||||
},
|
||||
}
|
||||
|
||||
for index, tc := range cases {
|
||||
actual := tc.health.IsHealthy(tc.lastTerm, tc.lastIndex, &tc.conf)
|
||||
if actual != tc.expected {
|
||||
t.Fatalf("bad value for case %d: %v", index, actual)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestServerHealth_IsStable(t *testing.T) {
|
||||
start := time.Now()
|
||||
cases := []struct {
|
||||
health *ServerHealth
|
||||
now time.Time
|
||||
conf Config
|
||||
expected bool
|
||||
}{
|
||||
// Healthy server, all values within allowed limits
|
||||
{
|
||||
health: &ServerHealth{Healthy: true, StableSince: start},
|
||||
now: start.Add(15 * time.Second),
|
||||
conf: Config{ServerStabilizationTime: 10 * time.Second},
|
||||
expected: true,
|
||||
},
|
||||
// Unhealthy server
|
||||
{
|
||||
health: &ServerHealth{Healthy: false},
|
||||
expected: false,
|
||||
},
|
||||
// Healthy server, hasn't reached stabilization time
|
||||
{
|
||||
health: &ServerHealth{Healthy: true, StableSince: start},
|
||||
now: start.Add(5 * time.Second),
|
||||
conf: Config{ServerStabilizationTime: 10 * time.Second},
|
||||
expected: false,
|
||||
},
|
||||
// Nil struct
|
||||
{
|
||||
health: nil,
|
||||
expected: false,
|
||||
},
|
||||
}
|
||||
|
||||
for index, tc := range cases {
|
||||
actual := tc.health.IsStable(tc.now, &tc.conf)
|
||||
if actual != tc.expected {
|
||||
t.Fatalf("bad value for case %d: %v", index, actual)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -2,9 +2,15 @@
|
|||
|
||||
package consul
|
||||
|
||||
import "github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
import (
|
||||
"github.com/hashicorp/consul/agent/metadata"
|
||||
autopilot "github.com/hashicorp/raft-autopilot"
|
||||
)
|
||||
|
||||
func (s *Server) initAutopilot(config *Config) {
|
||||
apDelegate := &AutopilotDelegate{s}
|
||||
s.autopilot = autopilot.NewAutopilot(s.logger, apDelegate, config.AutopilotInterval, config.ServerHealthInterval)
|
||||
func (s *Server) autopilotPromoter() autopilot.Promoter {
|
||||
return autopilot.DefaultPromoter()
|
||||
}
|
||||
|
||||
func (_ *Server) autopilotServerExt(_ *metadata.Server) interface{} {
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -1,10 +1,12 @@
|
|||
package consul
|
||||
|
||||
import (
|
||||
"context"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/sdk/testutil/retry"
|
||||
"github.com/hashicorp/consul/testrpc"
|
||||
"github.com/hashicorp/raft"
|
||||
|
@ -18,28 +20,20 @@ func TestAutopilot_IdempotentShutdown(t *testing.T) {
|
|||
defer s1.Shutdown()
|
||||
retry.Run(t, func(r *retry.R) { r.Check(waitForLeader(s1)) })
|
||||
|
||||
s1.autopilot.Start()
|
||||
s1.autopilot.Start()
|
||||
s1.autopilot.Start()
|
||||
s1.autopilot.Stop()
|
||||
s1.autopilot.Stop()
|
||||
s1.autopilot.Stop()
|
||||
s1.autopilot.Start(context.Background())
|
||||
s1.autopilot.Start(context.Background())
|
||||
s1.autopilot.Start(context.Background())
|
||||
<-s1.autopilot.Stop()
|
||||
<-s1.autopilot.Stop()
|
||||
<-s1.autopilot.Stop()
|
||||
}
|
||||
|
||||
func TestAutopilot_CleanupDeadServer(t *testing.T) {
|
||||
t.Parallel()
|
||||
for i := 1; i <= 3; i++ {
|
||||
testCleanupDeadServer(t, i)
|
||||
}
|
||||
}
|
||||
|
||||
func testCleanupDeadServer(t *testing.T, raftVersion int) {
|
||||
dc := "dc1"
|
||||
conf := func(c *Config) {
|
||||
c.Datacenter = dc
|
||||
c.Bootstrap = false
|
||||
c.BootstrapExpect = 5
|
||||
c.RaftConfig.ProtocolVersion = raft.ProtocolVersion(raftVersion)
|
||||
}
|
||||
dir1, s1 := testServerWithConfig(t, conf)
|
||||
defer os.RemoveAll(dir1)
|
||||
|
@ -119,10 +113,19 @@ func testCleanupDeadServer(t *testing.T, raftVersion int) {
|
|||
}
|
||||
|
||||
func TestAutopilot_CleanupDeadNonvoter(t *testing.T) {
|
||||
dir1, s1 := testServer(t)
|
||||
dir1, s1 := testServerWithConfig(t, func(c *Config) {
|
||||
c.AutopilotConfig = &structs.AutopilotConfig{
|
||||
CleanupDeadServers: true,
|
||||
ServerStabilizationTime: 100 * time.Millisecond,
|
||||
}
|
||||
})
|
||||
defer os.RemoveAll(dir1)
|
||||
defer s1.Shutdown()
|
||||
|
||||
// we have to wait for autopilot to be running long enough for the server stabilization time
|
||||
// to kick in for this test to work.
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
dir2, s2 := testServerDCBootstrap(t, "dc1", false)
|
||||
defer os.RemoveAll(dir2)
|
||||
defer s2.Shutdown()
|
||||
|
@ -316,7 +319,7 @@ func TestAutopilot_CleanupStaleRaftServer(t *testing.T) {
|
|||
}
|
||||
|
||||
// Verify we have 4 peers
|
||||
peers, err := s1.numPeers()
|
||||
peers, err := s1.autopilot.NumVoters()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -335,7 +338,6 @@ func TestAutopilot_PromoteNonVoter(t *testing.T) {
|
|||
dir1, s1 := testServerWithConfig(t, func(c *Config) {
|
||||
c.Datacenter = "dc1"
|
||||
c.Bootstrap = true
|
||||
c.RaftConfig.ProtocolVersion = 3
|
||||
c.AutopilotConfig.ServerStabilizationTime = 200 * time.Millisecond
|
||||
c.ServerHealthInterval = 100 * time.Millisecond
|
||||
c.AutopilotInterval = 100 * time.Millisecond
|
||||
|
@ -346,6 +348,10 @@ func TestAutopilot_PromoteNonVoter(t *testing.T) {
|
|||
defer codec.Close()
|
||||
testrpc.WaitForLeader(t, s1.RPC, "dc1")
|
||||
|
||||
// this may seem arbitrary but we need to get past the server stabilization time
|
||||
// so that we start factoring in that time for newly connected nodes.
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
dir2, s2 := testServerWithConfig(t, func(c *Config) {
|
||||
c.Datacenter = "dc1"
|
||||
c.Bootstrap = false
|
||||
|
@ -370,7 +376,7 @@ func TestAutopilot_PromoteNonVoter(t *testing.T) {
|
|||
if servers[1].Suffrage != raft.Nonvoter {
|
||||
r.Fatalf("bad: %v", servers)
|
||||
}
|
||||
health := s1.autopilot.GetServerHealth(string(servers[1].ID))
|
||||
health := s1.autopilot.GetServerHealth(servers[1].ID)
|
||||
if health == nil {
|
||||
r.Fatal("nil health")
|
||||
}
|
||||
|
@ -406,7 +412,6 @@ func TestAutopilot_MinQuorum(t *testing.T) {
|
|||
c.Bootstrap = false
|
||||
c.BootstrapExpect = 4
|
||||
c.AutopilotConfig.MinQuorum = 3
|
||||
c.RaftConfig.ProtocolVersion = raft.ProtocolVersion(2)
|
||||
c.AutopilotInterval = 100 * time.Millisecond
|
||||
}
|
||||
dir1, s1 := testServerWithConfig(t, conf)
|
||||
|
|
|
@ -12,7 +12,6 @@ import (
|
|||
"golang.org/x/time/rate"
|
||||
|
||||
"github.com/hashicorp/consul/agent/checks"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
libserf "github.com/hashicorp/consul/lib/serf"
|
||||
"github.com/hashicorp/consul/tlsutil"
|
||||
|
@ -438,7 +437,7 @@ type Config struct {
|
|||
|
||||
// AutopilotConfig is used to apply the initial autopilot config when
|
||||
// bootstrapping.
|
||||
AutopilotConfig *autopilot.Config
|
||||
AutopilotConfig *structs.AutopilotConfig
|
||||
|
||||
// ServerHealthInterval is the frequency with which the health of the
|
||||
// servers in the cluster will be updated.
|
||||
|
@ -590,7 +589,7 @@ func DefaultConfig() *Config {
|
|||
|
||||
// TODO (slackpad) - Until #3744 is done, we need to keep these
|
||||
// in sync with agent/config/default.go.
|
||||
AutopilotConfig: &autopilot.Config{
|
||||
AutopilotConfig: &structs.AutopilotConfig{
|
||||
CleanupDeadServers: true,
|
||||
LastContactThreshold: 200 * time.Millisecond,
|
||||
MaxTrailingLogs: 250,
|
||||
|
|
|
@ -10,7 +10,6 @@ import (
|
|||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/api"
|
||||
"github.com/hashicorp/consul/sdk/testutil"
|
||||
|
@ -1163,7 +1162,7 @@ func TestFSM_Autopilot(t *testing.T) {
|
|||
// Set the autopilot config using a request.
|
||||
req := structs.AutopilotSetConfigRequest{
|
||||
Datacenter: "dc1",
|
||||
Config: autopilot.Config{
|
||||
Config: structs.AutopilotConfig{
|
||||
CleanupDeadServers: true,
|
||||
LastContactThreshold: 10 * time.Second,
|
||||
MaxTrailingLogs: 300,
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package fsm
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/go-msgpack/codec"
|
||||
|
@ -605,7 +604,7 @@ func restorePreparedQuery(header *SnapshotHeader, restore *state.Restore, decode
|
|||
}
|
||||
|
||||
func restoreAutopilot(header *SnapshotHeader, restore *state.Restore, decoder *codec.Decoder) error {
|
||||
var req autopilot.Config
|
||||
var req structs.AutopilotConfig
|
||||
if err := decoder.Decode(&req); err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -7,7 +7,6 @@ import (
|
|||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/connect"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/api"
|
||||
|
@ -168,7 +167,7 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
|
|||
}
|
||||
require.NoError(t, fsm.state.PreparedQuerySet(14, &query))
|
||||
|
||||
autopilotConf := &autopilot.Config{
|
||||
autopilotConf := &structs.AutopilotConfig{
|
||||
CleanupDeadServers: true,
|
||||
LastContactThreshold: 100 * time.Millisecond,
|
||||
MaxTrailingLogs: 222,
|
||||
|
|
|
@ -38,7 +38,7 @@ func waitForLeader(servers ...*Server) error {
|
|||
// wantPeers determines whether the server has the given
|
||||
// number of voting raft peers.
|
||||
func wantPeers(s *Server, peers int) error {
|
||||
n, err := s.numPeers()
|
||||
n, err := s.autopilot.NumVoters()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -12,7 +12,6 @@ import (
|
|||
|
||||
"github.com/armon/go-metrics"
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/metadata"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/api"
|
||||
|
@ -37,10 +36,6 @@ var (
|
|||
// caRootPruneInterval is how often we check for stale CARoots to remove.
|
||||
caRootPruneInterval = time.Hour
|
||||
|
||||
// minAutopilotVersion is the minimum Consul version in which Autopilot features
|
||||
// are supported.
|
||||
minAutopilotVersion = version.Must(version.NewVersion("0.8.0"))
|
||||
|
||||
// minCentralizedConfigVersion is the minimum Consul version in which centralized
|
||||
// config is supported
|
||||
minCentralizedConfigVersion = version.Must(version.NewVersion("1.5.0"))
|
||||
|
@ -151,6 +146,8 @@ func (s *Server) leadershipTransfer() error {
|
|||
// leaderLoop runs as long as we are the leader to run various
|
||||
// maintenance activities
|
||||
func (s *Server) leaderLoop(stopCh chan struct{}) {
|
||||
stopCtx := &lib.StopChannelContext{StopCh: stopCh}
|
||||
|
||||
// Fire a user event indicating a new leader
|
||||
payload := []byte(s.config.NodeName)
|
||||
for name, segment := range s.LANSegments() {
|
||||
|
@ -183,7 +180,7 @@ RECONCILE:
|
|||
|
||||
// Check if we need to handle initial leadership actions
|
||||
if !establishedLeader {
|
||||
if err := s.establishLeadership(); err != nil {
|
||||
if err := s.establishLeadership(stopCtx); err != nil {
|
||||
s.logger.Error("failed to establish leadership", "error", err)
|
||||
// Immediately revoke leadership since we didn't successfully
|
||||
// establish leadership.
|
||||
|
@ -256,7 +253,7 @@ WAIT:
|
|||
// leader, which means revokeLeadership followed by an
|
||||
// establishLeadership().
|
||||
s.revokeLeadership()
|
||||
err := s.establishLeadership()
|
||||
err := s.establishLeadership(stopCtx)
|
||||
errCh <- err
|
||||
|
||||
// in case establishLeadership failed, we will try to
|
||||
|
@ -289,7 +286,7 @@ WAIT:
|
|||
// to invoke an initial barrier. The barrier is used to ensure any
|
||||
// previously inflight transactions have been committed and that our
|
||||
// state is up-to-date.
|
||||
func (s *Server) establishLeadership() error {
|
||||
func (s *Server) establishLeadership(ctx context.Context) error {
|
||||
start := time.Now()
|
||||
// check for the upgrade here - this helps us transition to new ACLs much
|
||||
// quicker if this is a new cluster or this is a test agent
|
||||
|
@ -328,7 +325,7 @@ func (s *Server) establishLeadership() error {
|
|||
}
|
||||
|
||||
s.getOrCreateAutopilotConfig()
|
||||
s.autopilot.Start()
|
||||
s.autopilot.Start(ctx)
|
||||
|
||||
// todo(kyhavlov): start a goroutine here for handling periodic CA rotation
|
||||
if err := s.initializeCA(); err != nil {
|
||||
|
@ -978,7 +975,7 @@ func (s *Server) stopFederationStateReplication() {
|
|||
}
|
||||
|
||||
// getOrCreateAutopilotConfig is used to get the autopilot config, initializing it if necessary
|
||||
func (s *Server) getOrCreateAutopilotConfig() *autopilot.Config {
|
||||
func (s *Server) getOrCreateAutopilotConfig() *structs.AutopilotConfig {
|
||||
logger := s.loggers.Named(logging.Autopilot)
|
||||
state := s.fsm.State()
|
||||
_, config, err := state.AutopilotConfig()
|
||||
|
@ -990,11 +987,6 @@ func (s *Server) getOrCreateAutopilotConfig() *autopilot.Config {
|
|||
return config
|
||||
}
|
||||
|
||||
if ok, _ := ServersInDCMeetMinimumVersion(s, s.config.Datacenter, minAutopilotVersion); !ok {
|
||||
logger.Warn("can't initialize until all servers are >= " + minAutopilotVersion.String())
|
||||
return nil
|
||||
}
|
||||
|
||||
config = s.config.AutopilotConfig
|
||||
req := structs.AutopilotSetConfigRequest{Config: *config}
|
||||
if _, err = s.raftApply(structs.AutopilotRequestType, req); err != nil {
|
||||
|
@ -1362,8 +1354,8 @@ func (s *Server) handleDeregisterMember(reason string, member serf.Member) error
|
|||
}
|
||||
|
||||
// Remove from Raft peers if this was a server
|
||||
if valid, parts := metadata.IsConsulServer(member); valid {
|
||||
if err := s.removeConsulServer(member, parts.Port); err != nil {
|
||||
if valid, _ := metadata.IsConsulServer(member); valid {
|
||||
if err := s.removeConsulServer(member); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
@ -1405,135 +1397,31 @@ func (s *Server) joinConsulServer(m serf.Member, parts *metadata.Server) error {
|
|||
}
|
||||
}
|
||||
|
||||
// Processing ourselves could result in trying to remove ourselves to
|
||||
// fix up our address, which would make us step down. This is only
|
||||
// safe to attempt if there are multiple servers available.
|
||||
configFuture := s.raft.GetConfiguration()
|
||||
if err := configFuture.Error(); err != nil {
|
||||
s.logger.Error("failed to get raft configuration", "error", err)
|
||||
return err
|
||||
}
|
||||
if m.Name == s.config.NodeName {
|
||||
if l := len(configFuture.Configuration().Servers); l < 3 {
|
||||
s.logger.Debug("Skipping self join check for node since the cluster is too small", "node", m.Name)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
// We used to do a check here and prevent adding the server if the cluster size was too small (1 or 2 servers) as a means
|
||||
// of preventing the case where we may remove ourselves and cause a loss of leadership. The Autopilot AddServer function
|
||||
// will now handle simple address updates better and so long as the address doesn't conflict with another node
|
||||
// it will not require a removal but will instead just update the address. If it would require a removal of other nodes
|
||||
// due to conflicts then the logic regarding cluster sizes will kick in and prevent doing anything dangerous that could
|
||||
// cause loss of leadership.
|
||||
|
||||
// See if it's already in the configuration. It's harmless to re-add it
|
||||
// but we want to avoid doing that if possible to prevent useless Raft
|
||||
// log entries. If the address is the same but the ID changed, remove the
|
||||
// old server before adding the new one.
|
||||
addr := (&net.TCPAddr{IP: m.Addr, Port: parts.Port}).String()
|
||||
minRaftProtocol, err := s.autopilot.MinRaftProtocol()
|
||||
// get the autpilot library version of a server from the serf member
|
||||
apServer, err := s.autopilotServer(m)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, server := range configFuture.Configuration().Servers {
|
||||
// No-op if the raft version is too low
|
||||
if server.Address == raft.ServerAddress(addr) && (minRaftProtocol < 2 || parts.RaftVersion < 3) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// If the address or ID matches an existing server, see if we need to remove the old one first
|
||||
if server.Address == raft.ServerAddress(addr) || server.ID == raft.ServerID(parts.ID) {
|
||||
// Exit with no-op if this is being called on an existing server
|
||||
if server.Address == raft.ServerAddress(addr) && server.ID == raft.ServerID(parts.ID) {
|
||||
return nil
|
||||
}
|
||||
future := s.raft.RemoveServer(server.ID, 0, 0)
|
||||
if server.Address == raft.ServerAddress(addr) {
|
||||
if err := future.Error(); err != nil {
|
||||
return fmt.Errorf("error removing server with duplicate address %q: %s", server.Address, err)
|
||||
}
|
||||
s.logger.Info("removed server with duplicate address", "address", server.Address)
|
||||
} else {
|
||||
if err := future.Error(); err != nil {
|
||||
return fmt.Errorf("error removing server with duplicate ID %q: %s", server.ID, err)
|
||||
}
|
||||
s.logger.Info("removed server with duplicate ID", "id", server.ID)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Attempt to add as a peer
|
||||
switch {
|
||||
case minRaftProtocol >= 3:
|
||||
addFuture := s.raft.AddNonvoter(raft.ServerID(parts.ID), raft.ServerAddress(addr), 0, 0)
|
||||
if err := addFuture.Error(); err != nil {
|
||||
s.logger.Error("failed to add raft peer", "error", err)
|
||||
return err
|
||||
}
|
||||
case minRaftProtocol == 2 && parts.RaftVersion >= 3:
|
||||
addFuture := s.raft.AddVoter(raft.ServerID(parts.ID), raft.ServerAddress(addr), 0, 0)
|
||||
if err := addFuture.Error(); err != nil {
|
||||
s.logger.Error("failed to add raft peer", "error", err)
|
||||
return err
|
||||
}
|
||||
default:
|
||||
addFuture := s.raft.AddPeer(raft.ServerAddress(addr))
|
||||
if err := addFuture.Error(); err != nil {
|
||||
s.logger.Error("failed to add raft peer", "error", err)
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Trigger a check to remove dead servers
|
||||
s.autopilot.RemoveDeadServers()
|
||||
|
||||
return nil
|
||||
// now ask autopilot to add it
|
||||
return s.autopilot.AddServer(apServer)
|
||||
}
|
||||
|
||||
// removeConsulServer is used to try to remove a consul server that has left
|
||||
func (s *Server) removeConsulServer(m serf.Member, port int) error {
|
||||
addr := (&net.TCPAddr{IP: m.Addr, Port: port}).String()
|
||||
|
||||
// See if it's already in the configuration. It's harmless to re-remove it
|
||||
// but we want to avoid doing that if possible to prevent useless Raft
|
||||
// log entries.
|
||||
configFuture := s.raft.GetConfiguration()
|
||||
if err := configFuture.Error(); err != nil {
|
||||
s.logger.Error("failed to get raft configuration", "error", err)
|
||||
func (s *Server) removeConsulServer(m serf.Member) error {
|
||||
server, err := s.autopilotServer(m)
|
||||
if err != nil || server == nil {
|
||||
return err
|
||||
}
|
||||
|
||||
minRaftProtocol, err := s.autopilot.MinRaftProtocol()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
_, parts := metadata.IsConsulServer(m)
|
||||
|
||||
// Pick which remove API to use based on how the server was added.
|
||||
for _, server := range configFuture.Configuration().Servers {
|
||||
// If we understand the new add/remove APIs and the server was added by ID, use the new remove API
|
||||
if minRaftProtocol >= 2 && server.ID == raft.ServerID(parts.ID) {
|
||||
s.logger.Info("removing server by ID", "id", server.ID)
|
||||
future := s.raft.RemoveServer(raft.ServerID(parts.ID), 0, 0)
|
||||
if err := future.Error(); err != nil {
|
||||
s.logger.Error("failed to remove raft peer",
|
||||
"id", server.ID,
|
||||
"error", err,
|
||||
)
|
||||
return err
|
||||
}
|
||||
break
|
||||
} else if server.Address == raft.ServerAddress(addr) {
|
||||
// If not, use the old remove API
|
||||
s.logger.Info("removing server by address", "address", server.Address)
|
||||
future := s.raft.RemovePeer(raft.ServerAddress(addr))
|
||||
if err := future.Error(); err != nil {
|
||||
s.logger.Error("failed to remove raft peer",
|
||||
"address", addr,
|
||||
"error", err,
|
||||
)
|
||||
return err
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
return s.autopilot.RemoveServer(server.ID)
|
||||
}
|
||||
|
||||
// reapTombstones is invoked by the current leader to manage garbage
|
||||
|
|
|
@ -741,7 +741,7 @@ func TestLeader_MultiBootstrap(t *testing.T) {
|
|||
|
||||
// Ensure we don't have multiple raft peers
|
||||
for _, s := range servers {
|
||||
peers, _ := s.numPeers()
|
||||
peers, _ := s.autopilot.NumVoters()
|
||||
if peers != 1 {
|
||||
t.Fatalf("should only have 1 raft peer!")
|
||||
}
|
||||
|
@ -886,7 +886,6 @@ func TestLeader_RollRaftServer(t *testing.T) {
|
|||
dir1, s1 := testServerWithConfig(t, func(c *Config) {
|
||||
c.Bootstrap = true
|
||||
c.Datacenter = "dc1"
|
||||
c.RaftConfig.ProtocolVersion = 2
|
||||
})
|
||||
defer os.RemoveAll(dir1)
|
||||
defer s1.Shutdown()
|
||||
|
@ -894,7 +893,6 @@ func TestLeader_RollRaftServer(t *testing.T) {
|
|||
dir2, s2 := testServerWithConfig(t, func(c *Config) {
|
||||
c.Bootstrap = false
|
||||
c.Datacenter = "dc1"
|
||||
c.RaftConfig.ProtocolVersion = 1
|
||||
})
|
||||
defer os.RemoveAll(dir2)
|
||||
defer s2.Shutdown()
|
||||
|
@ -902,7 +900,6 @@ func TestLeader_RollRaftServer(t *testing.T) {
|
|||
dir3, s3 := testServerWithConfig(t, func(c *Config) {
|
||||
c.Bootstrap = false
|
||||
c.Datacenter = "dc1"
|
||||
c.RaftConfig.ProtocolVersion = 2
|
||||
})
|
||||
defer os.RemoveAll(dir3)
|
||||
defer s3.Shutdown()
|
||||
|
@ -922,21 +919,15 @@ func TestLeader_RollRaftServer(t *testing.T) {
|
|||
|
||||
for _, s := range []*Server{s1, s3} {
|
||||
retry.Run(t, func(r *retry.R) {
|
||||
minVer, err := s.autopilot.MinRaftProtocol()
|
||||
if err != nil {
|
||||
r.Fatal(err)
|
||||
}
|
||||
if got, want := minVer, 2; got != want {
|
||||
r.Fatalf("got min raft version %d want %d", got, want)
|
||||
}
|
||||
// autopilot should force removal of the shutdown node
|
||||
r.Check(wantPeers(s, 2))
|
||||
})
|
||||
}
|
||||
|
||||
// Replace the dead server with one running raft protocol v3
|
||||
// Replace the dead server with a new one
|
||||
dir4, s4 := testServerWithConfig(t, func(c *Config) {
|
||||
c.Bootstrap = false
|
||||
c.Datacenter = "dc1"
|
||||
c.RaftConfig.ProtocolVersion = 3
|
||||
})
|
||||
defer os.RemoveAll(dir4)
|
||||
defer s4.Shutdown()
|
||||
|
@ -946,25 +937,7 @@ func TestLeader_RollRaftServer(t *testing.T) {
|
|||
// Make sure the dead server is removed and we're back to 3 total peers
|
||||
for _, s := range servers {
|
||||
retry.Run(t, func(r *retry.R) {
|
||||
addrs := 0
|
||||
ids := 0
|
||||
future := s.raft.GetConfiguration()
|
||||
if err := future.Error(); err != nil {
|
||||
r.Fatal(err)
|
||||
}
|
||||
for _, server := range future.Configuration().Servers {
|
||||
if string(server.ID) == string(server.Address) {
|
||||
addrs++
|
||||
} else {
|
||||
ids++
|
||||
}
|
||||
}
|
||||
if got, want := addrs, 2; got != want {
|
||||
r.Fatalf("got %d server addresses want %d", got, want)
|
||||
}
|
||||
if got, want := ids, 1; got != want {
|
||||
r.Fatalf("got %d server ids want %d", got, want)
|
||||
}
|
||||
r.Check(wantPeers(s, 3))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
|
@ -4,12 +4,13 @@ import (
|
|||
"fmt"
|
||||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
autopilot "github.com/hashicorp/raft-autopilot"
|
||||
"github.com/hashicorp/serf/serf"
|
||||
)
|
||||
|
||||
// AutopilotGetConfiguration is used to retrieve the current Autopilot configuration.
|
||||
func (op *Operator) AutopilotGetConfiguration(args *structs.DCSpecificRequest, reply *autopilot.Config) error {
|
||||
func (op *Operator) AutopilotGetConfiguration(args *structs.DCSpecificRequest, reply *structs.AutopilotConfig) error {
|
||||
if done, err := op.srv.ForwardRPC("Operator.AutopilotGetConfiguration", args, args, reply); done {
|
||||
return err
|
||||
}
|
||||
|
@ -23,7 +24,7 @@ func (op *Operator) AutopilotGetConfiguration(args *structs.DCSpecificRequest, r
|
|||
return err
|
||||
}
|
||||
if rule != nil && rule.OperatorRead(nil) != acl.Allow {
|
||||
return acl.ErrPermissionDenied
|
||||
return acl.PermissionDenied("Missing operator:read permissions")
|
||||
}
|
||||
|
||||
state := op.srv.fsm.State()
|
||||
|
@ -55,7 +56,7 @@ func (op *Operator) AutopilotSetConfiguration(args *structs.AutopilotSetConfigRe
|
|||
return err
|
||||
}
|
||||
if rule != nil && rule.OperatorWrite(nil) != acl.Allow {
|
||||
return acl.ErrPermissionDenied
|
||||
return acl.PermissionDenied("Missing operator:write permissions")
|
||||
}
|
||||
|
||||
// Apply the update
|
||||
|
@ -76,7 +77,7 @@ func (op *Operator) AutopilotSetConfiguration(args *structs.AutopilotSetConfigRe
|
|||
}
|
||||
|
||||
// ServerHealth is used to get the current health of the servers.
|
||||
func (op *Operator) ServerHealth(args *structs.DCSpecificRequest, reply *autopilot.OperatorHealthReply) error {
|
||||
func (op *Operator) ServerHealth(args *structs.DCSpecificRequest, reply *structs.AutopilotHealthReply) error {
|
||||
// This must be sent to the leader, so we fix the args since we are
|
||||
// re-using a structure where we don't support all the options.
|
||||
args.RequireConsistent = true
|
||||
|
@ -94,19 +95,75 @@ func (op *Operator) ServerHealth(args *structs.DCSpecificRequest, reply *autopil
|
|||
return err
|
||||
}
|
||||
if rule != nil && rule.OperatorRead(nil) != acl.Allow {
|
||||
return acl.ErrPermissionDenied
|
||||
return acl.PermissionDenied("Missing operator:read permissions")
|
||||
}
|
||||
|
||||
// Exit early if the min Raft version is too low
|
||||
minRaftProtocol, err := op.srv.autopilot.MinRaftProtocol()
|
||||
if err != nil {
|
||||
return fmt.Errorf("error getting server raft protocol versions: %s", err)
|
||||
}
|
||||
if minRaftProtocol < 3 {
|
||||
return fmt.Errorf("all servers must have raft_protocol set to 3 or higher to use this endpoint")
|
||||
state := op.srv.autopilot.GetState()
|
||||
|
||||
health := structs.AutopilotHealthReply{
|
||||
Healthy: state.Healthy,
|
||||
FailureTolerance: state.FailureTolerance,
|
||||
}
|
||||
|
||||
*reply = op.srv.autopilot.GetClusterHealth()
|
||||
for _, srv := range state.Servers {
|
||||
srvHealth := structs.AutopilotServerHealth{
|
||||
ID: string(srv.Server.ID),
|
||||
Name: srv.Server.Name,
|
||||
Address: string(srv.Server.Address),
|
||||
Version: srv.Server.Version,
|
||||
Leader: srv.State == autopilot.RaftLeader,
|
||||
Voter: srv.State == autopilot.RaftLeader || srv.State == autopilot.RaftVoter,
|
||||
LastContact: srv.Stats.LastContact,
|
||||
LastTerm: srv.Stats.LastTerm,
|
||||
LastIndex: srv.Stats.LastIndex,
|
||||
Healthy: srv.Health.Healthy,
|
||||
StableSince: srv.Health.StableSince,
|
||||
}
|
||||
|
||||
switch srv.Server.NodeStatus {
|
||||
case autopilot.NodeAlive:
|
||||
srvHealth.SerfStatus = serf.StatusAlive
|
||||
case autopilot.NodeLeft:
|
||||
srvHealth.SerfStatus = serf.StatusLeft
|
||||
case autopilot.NodeFailed:
|
||||
srvHealth.SerfStatus = serf.StatusFailed
|
||||
default:
|
||||
srvHealth.SerfStatus = serf.StatusNone
|
||||
}
|
||||
|
||||
health.Servers = append(health.Servers, srvHealth)
|
||||
}
|
||||
|
||||
*reply = health
|
||||
return nil
|
||||
}
|
||||
|
||||
func (op *Operator) AutopilotState(args *structs.DCSpecificRequest, reply *autopilot.State) error {
|
||||
// This must be sent to the leader, so we fix the args since we are
|
||||
// re-using a structure where we don't support all the options.
|
||||
args.RequireConsistent = true
|
||||
args.AllowStale = false
|
||||
if done, err := op.srv.ForwardRPC("Operator.AutopilotState", args, args, reply); done {
|
||||
return err
|
||||
}
|
||||
|
||||
// This action requires operator read access.
|
||||
identity, rule, err := op.srv.ResolveTokenToIdentityAndAuthorizer(args.Token)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err := op.srv.validateEnterpriseToken(identity); err != nil {
|
||||
return err
|
||||
}
|
||||
if rule != nil && rule.OperatorRead(nil) != acl.Allow {
|
||||
return acl.PermissionDenied("Missing operator:read permissions")
|
||||
}
|
||||
|
||||
state := op.srv.autopilot.GetState()
|
||||
if state == nil {
|
||||
return fmt.Errorf("Failed to get autopilot state: no state found")
|
||||
}
|
||||
|
||||
*reply = *state
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -2,17 +2,17 @@ package consul
|
|||
|
||||
import (
|
||||
"os"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/sdk/testutil/retry"
|
||||
"github.com/hashicorp/consul/testrpc"
|
||||
msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc"
|
||||
"github.com/hashicorp/raft"
|
||||
autopilot "github.com/hashicorp/raft-autopilot"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestOperator_Autopilot_GetConfiguration(t *testing.T) {
|
||||
|
@ -30,7 +30,7 @@ func TestOperator_Autopilot_GetConfiguration(t *testing.T) {
|
|||
arg := structs.DCSpecificRequest{
|
||||
Datacenter: "dc1",
|
||||
}
|
||||
var reply autopilot.Config
|
||||
var reply structs.AutopilotConfig
|
||||
err := msgpackrpc.CallWithCodec(codec, "Operator.AutopilotGetConfiguration", &arg, &reply)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
|
@ -60,7 +60,7 @@ func TestOperator_Autopilot_GetConfiguration_ACLDeny(t *testing.T) {
|
|||
arg := structs.DCSpecificRequest{
|
||||
Datacenter: "dc1",
|
||||
}
|
||||
var reply autopilot.Config
|
||||
var reply structs.AutopilotConfig
|
||||
err := msgpackrpc.CallWithCodec(codec, "Operator.AutopilotGetConfiguration", &arg, &reply)
|
||||
if !acl.IsErrPermissionDenied(err) {
|
||||
t.Fatalf("err: %v", err)
|
||||
|
@ -114,7 +114,7 @@ func TestOperator_Autopilot_SetConfiguration(t *testing.T) {
|
|||
// Change the autopilot config from the default
|
||||
arg := structs.AutopilotSetConfigRequest{
|
||||
Datacenter: "dc1",
|
||||
Config: autopilot.Config{
|
||||
Config: structs.AutopilotConfig{
|
||||
CleanupDeadServers: true,
|
||||
MinQuorum: 3,
|
||||
},
|
||||
|
@ -155,7 +155,7 @@ func TestOperator_Autopilot_SetConfiguration_ACLDeny(t *testing.T) {
|
|||
// Try to set config without permissions
|
||||
arg := structs.AutopilotSetConfigRequest{
|
||||
Datacenter: "dc1",
|
||||
Config: autopilot.Config{
|
||||
Config: structs.AutopilotConfig{
|
||||
CleanupDeadServers: true,
|
||||
},
|
||||
}
|
||||
|
@ -236,7 +236,7 @@ func TestOperator_ServerHealth(t *testing.T) {
|
|||
arg := structs.DCSpecificRequest{
|
||||
Datacenter: "dc1",
|
||||
}
|
||||
var reply autopilot.OperatorHealthReply
|
||||
var reply structs.AutopilotHealthReply
|
||||
err := msgpackrpc.CallWithCodec(codec, "Operator.ServerHealth", &arg, &reply)
|
||||
if err != nil {
|
||||
r.Fatalf("err: %v", err)
|
||||
|
@ -263,24 +263,52 @@ func TestOperator_ServerHealth(t *testing.T) {
|
|||
})
|
||||
}
|
||||
|
||||
func TestOperator_ServerHealth_UnsupportedRaftVersion(t *testing.T) {
|
||||
func TestOperator_AutopilotState(t *testing.T) {
|
||||
t.Parallel()
|
||||
dir1, s1 := testServerWithConfig(t, func(c *Config) {
|
||||
conf := func(c *Config) {
|
||||
c.Datacenter = "dc1"
|
||||
c.Bootstrap = true
|
||||
c.RaftConfig.ProtocolVersion = 2
|
||||
})
|
||||
c.Bootstrap = false
|
||||
c.BootstrapExpect = 3
|
||||
c.RaftConfig.ProtocolVersion = 3
|
||||
c.ServerHealthInterval = 100 * time.Millisecond
|
||||
c.AutopilotInterval = 100 * time.Millisecond
|
||||
}
|
||||
dir1, s1 := testServerWithConfig(t, conf)
|
||||
defer os.RemoveAll(dir1)
|
||||
defer s1.Shutdown()
|
||||
codec := rpcClient(t, s1)
|
||||
defer codec.Close()
|
||||
|
||||
arg := structs.DCSpecificRequest{
|
||||
Datacenter: "dc1",
|
||||
}
|
||||
var reply autopilot.OperatorHealthReply
|
||||
err := msgpackrpc.CallWithCodec(codec, "Operator.ServerHealth", &arg, &reply)
|
||||
if err == nil || !strings.Contains(err.Error(), "raft_protocol set to 3 or higher") {
|
||||
t.Fatalf("bad: %v", err)
|
||||
}
|
||||
dir2, s2 := testServerWithConfig(t, conf)
|
||||
defer os.RemoveAll(dir2)
|
||||
defer s2.Shutdown()
|
||||
joinLAN(t, s2, s1)
|
||||
|
||||
dir3, s3 := testServerWithConfig(t, conf)
|
||||
defer os.RemoveAll(dir3)
|
||||
defer s3.Shutdown()
|
||||
joinLAN(t, s3, s1)
|
||||
|
||||
testrpc.WaitForLeader(t, s1.RPC, "dc1")
|
||||
retry.Run(t, func(r *retry.R) {
|
||||
arg := structs.DCSpecificRequest{
|
||||
Datacenter: "dc1",
|
||||
}
|
||||
var reply autopilot.State
|
||||
err := msgpackrpc.CallWithCodec(codec, "Operator.AutopilotState", &arg, &reply)
|
||||
require.NoError(r, err)
|
||||
require.True(r, reply.Healthy)
|
||||
require.Equal(r, 1, reply.FailureTolerance)
|
||||
require.Len(r, reply.Servers, 3)
|
||||
|
||||
// Leader should have LastContact == 0, others should be positive
|
||||
for _, s := range reply.Servers {
|
||||
isLeader := s1.raft.Leader() == s.Server.Address
|
||||
if isLeader {
|
||||
require.Zero(r, s.Stats.LastContact)
|
||||
} else {
|
||||
require.NotEqual(r, time.Duration(0), s.Stats.LastContact)
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
|
|
@ -110,34 +110,15 @@ func (op *Operator) RaftRemovePeerByAddress(args *structs.RaftRemovePeerRequest,
|
|||
}
|
||||
|
||||
REMOVE:
|
||||
// The Raft library itself will prevent various forms of foot-shooting,
|
||||
// like making a configuration with no voters. Some consideration was
|
||||
// given here to adding more checks, but it was decided to make this as
|
||||
// low-level and direct as possible. We've got ACL coverage to lock this
|
||||
// down, and if you are an operator, it's assumed you know what you are
|
||||
// doing if you are calling this. If you remove a peer that's known to
|
||||
// Serf, for example, it will come back when the leader does a reconcile
|
||||
// pass.
|
||||
minRaftProtocol, err := op.srv.autopilot.MinRaftProtocol()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
var future raft.Future
|
||||
if minRaftProtocol >= 2 {
|
||||
future = op.srv.raft.RemoveServer(args.ID, 0, 0)
|
||||
} else {
|
||||
future = op.srv.raft.RemovePeer(args.Address)
|
||||
}
|
||||
if err := future.Error(); err != nil {
|
||||
op.logger.Warn("Failed to remove Raft peer",
|
||||
"peer", args.Address,
|
||||
if err := op.srv.autopilot.RemoveServer(args.ID); err != nil {
|
||||
op.logger.Warn("Failed to remove Raft server",
|
||||
"address", args.Address,
|
||||
"error", err,
|
||||
)
|
||||
return err
|
||||
}
|
||||
|
||||
op.logger.Warn("Removed Raft peer", "peer", args.Address)
|
||||
op.logger.Warn("Removed Raft server", "address", args.Address)
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -190,18 +171,7 @@ REMOVE:
|
|||
// doing if you are calling this. If you remove a peer that's known to
|
||||
// Serf, for example, it will come back when the leader does a reconcile
|
||||
// pass.
|
||||
minRaftProtocol, err := op.srv.autopilot.MinRaftProtocol()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
var future raft.Future
|
||||
if minRaftProtocol >= 2 {
|
||||
future = op.srv.raft.RemoveServer(args.ID, 0, 0)
|
||||
} else {
|
||||
future = op.srv.raft.RemovePeer(args.Address)
|
||||
}
|
||||
if err := future.Error(); err != nil {
|
||||
if err := op.srv.autopilot.RemoveServer(args.ID); err != nil {
|
||||
op.logger.Warn("Failed to remove Raft peer with id",
|
||||
"peer_id", args.ID,
|
||||
"error", err,
|
||||
|
|
|
@ -23,6 +23,7 @@ import (
|
|||
"github.com/hashicorp/go-memdb"
|
||||
"github.com/hashicorp/memberlist"
|
||||
"github.com/hashicorp/raft"
|
||||
autopilot "github.com/hashicorp/raft-autopilot"
|
||||
raftboltdb "github.com/hashicorp/raft-boltdb"
|
||||
"github.com/hashicorp/serf/serf"
|
||||
"golang.org/x/time/rate"
|
||||
|
@ -32,7 +33,6 @@ import (
|
|||
ca "github.com/hashicorp/consul/agent/connect/ca"
|
||||
"github.com/hashicorp/consul/agent/consul/authmethod"
|
||||
"github.com/hashicorp/consul/agent/consul/authmethod/ssoauth"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/consul/fsm"
|
||||
"github.com/hashicorp/consul/agent/consul/state"
|
||||
"github.com/hashicorp/consul/agent/consul/usagemetrics"
|
||||
|
@ -137,9 +137,6 @@ type Server struct {
|
|||
// autopilot is the Autopilot instance for this server.
|
||||
autopilot *autopilot.Autopilot
|
||||
|
||||
// autopilotWaitGroup is used to block until Autopilot shuts down.
|
||||
autopilotWaitGroup sync.WaitGroup
|
||||
|
||||
// caProviderReconfigurationLock guards the provider reconfiguration.
|
||||
caProviderReconfigurationLock sync.Mutex
|
||||
// caProvider is the current CA provider in use for Connect. This is
|
||||
|
@ -993,7 +990,7 @@ func (s *Server) Leave() error {
|
|||
s.logger.Info("server starting leave")
|
||||
|
||||
// Check the number of known peers
|
||||
numPeers, err := s.numPeers()
|
||||
numPeers, err := s.autopilot.NumVoters()
|
||||
if err != nil {
|
||||
s.logger.Error("failed to check raft peers", "error", err)
|
||||
return err
|
||||
|
@ -1007,21 +1004,8 @@ func (s *Server) Leave() error {
|
|||
// removed for some sane period of time.
|
||||
isLeader := s.IsLeader()
|
||||
if isLeader && numPeers > 1 {
|
||||
minRaftProtocol, err := s.autopilot.MinRaftProtocol()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if minRaftProtocol >= 2 && s.config.RaftConfig.ProtocolVersion >= 3 {
|
||||
future := s.raft.RemoveServer(raft.ServerID(s.config.NodeID), 0, 0)
|
||||
if err := future.Error(); err != nil {
|
||||
s.logger.Error("failed to remove ourself as raft peer", "error", err)
|
||||
}
|
||||
} else {
|
||||
future := s.raft.RemovePeer(addr)
|
||||
if err := future.Error(); err != nil {
|
||||
s.logger.Error("failed to remove ourself as raft peer", "error", err)
|
||||
}
|
||||
if err := s.autopilot.RemoveServer(raft.ServerID(s.config.NodeID)); err != nil {
|
||||
s.logger.Error("failed to remove ourself as a Raft peer", "error", err)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1100,18 +1084,6 @@ func (s *Server) Leave() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// numPeers is used to check on the number of known peers, including potentially
|
||||
// the local node. We count only voters, since others can't actually become
|
||||
// leader, so aren't considered peers.
|
||||
func (s *Server) numPeers() (int, error) {
|
||||
future := s.raft.GetConfiguration()
|
||||
if err := future.Error(); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
return autopilot.NumPeers(future.Configuration()), nil
|
||||
}
|
||||
|
||||
// JoinLAN is used to have Consul join the inner-DC pool
|
||||
// The target address should be another node inside the DC
|
||||
// listening on the Serf LAN address
|
||||
|
@ -1194,17 +1166,21 @@ func (s *Server) RemoveFailedNode(node string, prune bool) error {
|
|||
if err := removeFn(s.serfLAN, node); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
wanNode := node
|
||||
|
||||
// The Serf WAN pool stores members as node.datacenter
|
||||
// so the dc is appended if not present
|
||||
if !strings.HasSuffix(node, "."+s.config.Datacenter) {
|
||||
node = node + "." + s.config.Datacenter
|
||||
wanNode = node + "." + s.config.Datacenter
|
||||
}
|
||||
if s.serfWAN != nil {
|
||||
if err := removeFn(s.serfWAN, node); err != nil {
|
||||
if err := removeFn(s.serfWAN, wanNode); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
|
||||
return s.removeFailedNodeEnterprise(removeFn, node, wanNode)
|
||||
}
|
||||
|
||||
// IsLeader checks if this server is the cluster leader
|
||||
|
|
|
@ -1,24 +1,12 @@
|
|||
// +build !consulent
|
||||
|
||||
package consul
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/consul/logging"
|
||||
"github.com/hashicorp/serf/serf"
|
||||
)
|
||||
|
||||
func init() {
|
||||
registerEndpoint(func(s *Server) interface{} { return &ACL{s, s.loggers.Named(logging.ACL)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Catalog{s, s.loggers.Named(logging.Catalog)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return NewCoordinate(s, s.logger) })
|
||||
registerEndpoint(func(s *Server) interface{} { return &ConfigEntry{s} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &ConnectCA{srv: s, logger: s.loggers.Named(logging.Connect)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &FederationState{s} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &DiscoveryChain{s} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Health{s} })
|
||||
registerEndpoint(func(s *Server) interface{} { return NewIntentionEndpoint(s, s.loggers.Named(logging.Intentions)) })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Internal{s, s.loggers.Named(logging.Internal)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &KVS{s, s.loggers.Named(logging.KV)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Operator{s, s.loggers.Named(logging.Operator)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &PreparedQuery{s, s.loggers.Named(logging.PreparedQuery)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Session{s, s.loggers.Named(logging.Session)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Status{s} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Txn{s, s.loggers.Named(logging.Transaction)} })
|
||||
func (s *Server) removeFailedNodeEnterprise(remove func(*serf.Serf, string) error, node, wanNode string) error {
|
||||
// nothing to do for oss
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -0,0 +1,22 @@
|
|||
package consul
|
||||
|
||||
import "github.com/hashicorp/consul/logging"
|
||||
|
||||
func init() {
|
||||
registerEndpoint(func(s *Server) interface{} { return &ACL{s, s.loggers.Named(logging.ACL)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Catalog{s, s.loggers.Named(logging.Catalog)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return NewCoordinate(s, s.logger) })
|
||||
registerEndpoint(func(s *Server) interface{} { return &ConfigEntry{s} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &ConnectCA{srv: s, logger: s.loggers.Named(logging.Connect)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &FederationState{s} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &DiscoveryChain{s} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Health{s} })
|
||||
registerEndpoint(func(s *Server) interface{} { return NewIntentionEndpoint(s, s.loggers.Named(logging.Intentions)) })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Internal{s, s.loggers.Named(logging.Internal)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &KVS{s, s.loggers.Named(logging.KV)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Operator{s, s.loggers.Named(logging.Operator)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &PreparedQuery{s, s.loggers.Named(logging.PreparedQuery)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Session{s, s.loggers.Named(logging.Session)} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Status{s} })
|
||||
registerEndpoint(func(s *Server) interface{} { return &Txn{s, s.loggers.Named(logging.Transaction)} })
|
||||
}
|
|
@ -403,20 +403,12 @@ func (s *Server) maybeBootstrap() {
|
|||
// Attempt a live bootstrap!
|
||||
var configuration raft.Configuration
|
||||
var addrs []string
|
||||
minRaftVersion, err := s.autopilot.MinRaftProtocol()
|
||||
if err != nil {
|
||||
s.logger.Error("Failed to read server raft versions", "error", err)
|
||||
}
|
||||
|
||||
for _, server := range servers {
|
||||
addr := server.Addr.String()
|
||||
addrs = append(addrs, addr)
|
||||
var id raft.ServerID
|
||||
if minRaftVersion >= 3 {
|
||||
id = raft.ServerID(server.ID)
|
||||
} else {
|
||||
id = raft.ServerID(addr)
|
||||
}
|
||||
id := raft.ServerID(server.ID)
|
||||
|
||||
suffrage := raft.Voter
|
||||
if server.NonVoter {
|
||||
suffrage = raft.Nonvoter
|
||||
|
|
|
@ -3,7 +3,7 @@ package state
|
|||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/go-memdb"
|
||||
)
|
||||
|
||||
|
@ -30,13 +30,13 @@ func init() {
|
|||
}
|
||||
|
||||
// Autopilot is used to pull the autopilot config from the snapshot.
|
||||
func (s *Snapshot) Autopilot() (*autopilot.Config, error) {
|
||||
func (s *Snapshot) Autopilot() (*structs.AutopilotConfig, error) {
|
||||
c, err := s.tx.First("autopilot-config", "id")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
config, ok := c.(*autopilot.Config)
|
||||
config, ok := c.(*structs.AutopilotConfig)
|
||||
if !ok {
|
||||
return nil, nil
|
||||
}
|
||||
|
@ -45,7 +45,7 @@ func (s *Snapshot) Autopilot() (*autopilot.Config, error) {
|
|||
}
|
||||
|
||||
// Autopilot is used when restoring from a snapshot.
|
||||
func (s *Restore) Autopilot(config *autopilot.Config) error {
|
||||
func (s *Restore) Autopilot(config *structs.AutopilotConfig) error {
|
||||
if err := s.tx.Insert("autopilot-config", config); err != nil {
|
||||
return fmt.Errorf("failed restoring autopilot config: %s", err)
|
||||
}
|
||||
|
@ -54,7 +54,7 @@ func (s *Restore) Autopilot(config *autopilot.Config) error {
|
|||
}
|
||||
|
||||
// AutopilotConfig is used to get the current Autopilot configuration.
|
||||
func (s *Store) AutopilotConfig() (uint64, *autopilot.Config, error) {
|
||||
func (s *Store) AutopilotConfig() (uint64, *structs.AutopilotConfig, error) {
|
||||
tx := s.db.Txn(false)
|
||||
defer tx.Abort()
|
||||
|
||||
|
@ -64,7 +64,7 @@ func (s *Store) AutopilotConfig() (uint64, *autopilot.Config, error) {
|
|||
return 0, nil, fmt.Errorf("failed autopilot config lookup: %s", err)
|
||||
}
|
||||
|
||||
config, ok := c.(*autopilot.Config)
|
||||
config, ok := c.(*structs.AutopilotConfig)
|
||||
if !ok {
|
||||
return 0, nil, nil
|
||||
}
|
||||
|
@ -73,7 +73,7 @@ func (s *Store) AutopilotConfig() (uint64, *autopilot.Config, error) {
|
|||
}
|
||||
|
||||
// AutopilotSetConfig is used to set the current Autopilot configuration.
|
||||
func (s *Store) AutopilotSetConfig(idx uint64, config *autopilot.Config) error {
|
||||
func (s *Store) AutopilotSetConfig(idx uint64, config *structs.AutopilotConfig) error {
|
||||
tx := s.db.WriteTxn(idx)
|
||||
defer tx.Abort()
|
||||
|
||||
|
@ -87,7 +87,7 @@ func (s *Store) AutopilotSetConfig(idx uint64, config *autopilot.Config) error {
|
|||
// AutopilotCASConfig is used to try updating the Autopilot configuration with a
|
||||
// given Raft index. If the CAS index specified is not equal to the last observed index
|
||||
// for the config, then the call is a noop,
|
||||
func (s *Store) AutopilotCASConfig(idx, cidx uint64, config *autopilot.Config) (bool, error) {
|
||||
func (s *Store) AutopilotCASConfig(idx, cidx uint64, config *structs.AutopilotConfig) (bool, error) {
|
||||
tx := s.db.WriteTxn(idx)
|
||||
defer tx.Abort()
|
||||
|
||||
|
@ -100,7 +100,7 @@ func (s *Store) AutopilotCASConfig(idx, cidx uint64, config *autopilot.Config) (
|
|||
// If the existing index does not match the provided CAS
|
||||
// index arg, then we shouldn't update anything and can safely
|
||||
// return early here.
|
||||
e, ok := existing.(*autopilot.Config)
|
||||
e, ok := existing.(*structs.AutopilotConfig)
|
||||
if !ok || e.ModifyIndex != cidx {
|
||||
return false, nil
|
||||
}
|
||||
|
@ -113,7 +113,7 @@ func (s *Store) AutopilotCASConfig(idx, cidx uint64, config *autopilot.Config) (
|
|||
return err == nil, err
|
||||
}
|
||||
|
||||
func autopilotSetConfigTxn(tx *txn, idx uint64, config *autopilot.Config) error {
|
||||
func autopilotSetConfigTxn(tx *txn, idx uint64, config *structs.AutopilotConfig) error {
|
||||
// Check for an existing config
|
||||
existing, err := tx.First("autopilot-config", "id")
|
||||
if err != nil {
|
||||
|
@ -122,7 +122,7 @@ func autopilotSetConfigTxn(tx *txn, idx uint64, config *autopilot.Config) error
|
|||
|
||||
// Set the indexes.
|
||||
if existing != nil {
|
||||
config.CreateIndex = existing.(*autopilot.Config).CreateIndex
|
||||
config.CreateIndex = existing.(*structs.AutopilotConfig).CreateIndex
|
||||
} else {
|
||||
config.CreateIndex = idx
|
||||
}
|
||||
|
|
|
@ -5,14 +5,14 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestStateStore_Autopilot(t *testing.T) {
|
||||
s := testStateStore(t)
|
||||
|
||||
expected := &autopilot.Config{
|
||||
expected := &structs.AutopilotConfig{
|
||||
CleanupDeadServers: true,
|
||||
LastContactThreshold: 5 * time.Second,
|
||||
MaxTrailingLogs: 500,
|
||||
|
@ -41,7 +41,7 @@ func TestStateStore_Autopilot(t *testing.T) {
|
|||
func TestStateStore_AutopilotCAS(t *testing.T) {
|
||||
s := testStateStore(t)
|
||||
|
||||
expected := &autopilot.Config{
|
||||
expected := &structs.AutopilotConfig{
|
||||
CleanupDeadServers: true,
|
||||
}
|
||||
|
||||
|
@ -53,7 +53,7 @@ func TestStateStore_AutopilotCAS(t *testing.T) {
|
|||
}
|
||||
|
||||
// Do a CAS with an index lower than the entry
|
||||
ok, err := s.AutopilotCASConfig(2, 0, &autopilot.Config{
|
||||
ok, err := s.AutopilotCASConfig(2, 0, &structs.AutopilotConfig{
|
||||
CleanupDeadServers: false,
|
||||
})
|
||||
if ok || err != nil {
|
||||
|
@ -74,7 +74,7 @@ func TestStateStore_AutopilotCAS(t *testing.T) {
|
|||
}
|
||||
|
||||
// Do another CAS, this time with the correct index
|
||||
ok, err = s.AutopilotCASConfig(2, 1, &autopilot.Config{
|
||||
ok, err = s.AutopilotCASConfig(2, 1, &structs.AutopilotConfig{
|
||||
CleanupDeadServers: false,
|
||||
})
|
||||
if !ok || err != nil {
|
||||
|
@ -96,7 +96,7 @@ func TestStateStore_AutopilotCAS(t *testing.T) {
|
|||
|
||||
func TestStateStore_Autopilot_Snapshot_Restore(t *testing.T) {
|
||||
s := testStateStore(t)
|
||||
before := &autopilot.Config{
|
||||
before := &structs.AutopilotConfig{
|
||||
CleanupDeadServers: true,
|
||||
}
|
||||
if err := s.AutopilotSetConfig(99, before); err != nil {
|
||||
|
@ -106,7 +106,7 @@ func TestStateStore_Autopilot_Snapshot_Restore(t *testing.T) {
|
|||
snap := s.Snapshot()
|
||||
defer snap.Close()
|
||||
|
||||
after := &autopilot.Config{
|
||||
after := &structs.AutopilotConfig{
|
||||
CleanupDeadServers: false,
|
||||
}
|
||||
if err := s.AutopilotSetConfig(100, after); err != nil {
|
||||
|
|
|
@ -2,13 +2,14 @@ package consul
|
|||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"sync"
|
||||
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/metadata"
|
||||
"github.com/hashicorp/consul/agent/pool"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/serf/serf"
|
||||
"github.com/hashicorp/raft"
|
||||
autopilot "github.com/hashicorp/raft-autopilot"
|
||||
)
|
||||
|
||||
// StatsFetcher has two functions for autopilot. First, lets us fetch all the
|
||||
|
@ -22,7 +23,7 @@ type StatsFetcher struct {
|
|||
logger hclog.Logger
|
||||
pool *pool.ConnPool
|
||||
datacenter string
|
||||
inflight map[string]struct{}
|
||||
inflight map[raft.ServerID]struct{}
|
||||
inflightLock sync.Mutex
|
||||
}
|
||||
|
||||
|
@ -32,7 +33,7 @@ func NewStatsFetcher(logger hclog.Logger, pool *pool.ConnPool, datacenter string
|
|||
logger: logger,
|
||||
pool: pool,
|
||||
datacenter: datacenter,
|
||||
inflight: make(map[string]struct{}),
|
||||
inflight: make(map[raft.ServerID]struct{}),
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -40,36 +41,44 @@ func NewStatsFetcher(logger hclog.Logger, pool *pool.ConnPool, datacenter string
|
|||
// cancel this when the context is canceled because we only want one in-flight
|
||||
// RPC to each server, so we let it finish and then clean up the in-flight
|
||||
// tracking.
|
||||
func (f *StatsFetcher) fetch(server *metadata.Server, replyCh chan *autopilot.ServerStats) {
|
||||
func (f *StatsFetcher) fetch(server *autopilot.Server, replyCh chan *autopilot.ServerStats) {
|
||||
var args struct{}
|
||||
var reply autopilot.ServerStats
|
||||
err := f.pool.RPC(f.datacenter, server.ShortName, server.Addr, "Status.RaftStats", &args, &reply)
|
||||
var reply structs.RaftStats
|
||||
|
||||
// defer some cleanup to notify everything else that the fetching is no longer occurring
|
||||
// this is easier than trying to make the conditionals line up just right.
|
||||
defer func() {
|
||||
f.inflightLock.Lock()
|
||||
delete(f.inflight, server.ID)
|
||||
f.inflightLock.Unlock()
|
||||
}()
|
||||
|
||||
addr, err := net.ResolveTCPAddr("tcp", string(server.Address))
|
||||
if err != nil {
|
||||
f.logger.Warn("error resolving TCP address for server",
|
||||
"address", server.Address,
|
||||
"error", err)
|
||||
return
|
||||
}
|
||||
|
||||
err = f.pool.RPC(f.datacenter, server.Name, addr, "Status.RaftStats", &args, &reply)
|
||||
if err != nil {
|
||||
f.logger.Warn("error getting server health from server",
|
||||
"server", server.Name,
|
||||
"error", err,
|
||||
)
|
||||
} else {
|
||||
replyCh <- &reply
|
||||
return
|
||||
}
|
||||
|
||||
f.inflightLock.Lock()
|
||||
delete(f.inflight, server.ID)
|
||||
f.inflightLock.Unlock()
|
||||
replyCh <- reply.ToAutopilotServerStats()
|
||||
}
|
||||
|
||||
// Fetch will attempt to query all the servers in parallel.
|
||||
func (f *StatsFetcher) Fetch(ctx context.Context, members []serf.Member) map[string]*autopilot.ServerStats {
|
||||
func (f *StatsFetcher) Fetch(ctx context.Context, servers map[raft.ServerID]*autopilot.Server) map[raft.ServerID]*autopilot.ServerStats {
|
||||
type workItem struct {
|
||||
server *metadata.Server
|
||||
server *autopilot.Server
|
||||
replyCh chan *autopilot.ServerStats
|
||||
}
|
||||
var servers []*metadata.Server
|
||||
for _, s := range members {
|
||||
if ok, parts := metadata.IsConsulServer(s); ok {
|
||||
servers = append(servers, parts)
|
||||
}
|
||||
}
|
||||
|
||||
// Skip any servers that have inflight requests.
|
||||
var work []*workItem
|
||||
|
@ -94,7 +103,7 @@ func (f *StatsFetcher) Fetch(ctx context.Context, members []serf.Member) map[str
|
|||
|
||||
// Now wait for the results to come in, or for the context to be
|
||||
// canceled.
|
||||
replies := make(map[string]*autopilot.ServerStats)
|
||||
replies := make(map[raft.ServerID]*autopilot.ServerStats)
|
||||
for _, workItem := range work {
|
||||
// Drain the reply first if there is one.
|
||||
select {
|
||||
|
|
|
@ -10,6 +10,7 @@ import (
|
|||
"github.com/hashicorp/consul/sdk/testutil/retry"
|
||||
"github.com/hashicorp/consul/testrpc"
|
||||
"github.com/hashicorp/consul/types"
|
||||
"github.com/hashicorp/raft"
|
||||
)
|
||||
|
||||
func TestStatsFetcher(t *testing.T) {
|
||||
|
@ -48,7 +49,7 @@ func TestStatsFetcher(t *testing.T) {
|
|||
retry.Run(t, func(r *retry.R) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
|
||||
defer cancel()
|
||||
stats := s1.statsFetcher.Fetch(ctx, s1.LANMembers())
|
||||
stats := s1.statsFetcher.Fetch(ctx, s1.autopilotServers())
|
||||
if len(stats) != 3 {
|
||||
r.Fatalf("bad: %#v", stats)
|
||||
}
|
||||
|
@ -71,12 +72,12 @@ func TestStatsFetcher(t *testing.T) {
|
|||
// from it.
|
||||
func() {
|
||||
retry.Run(t, func(r *retry.R) {
|
||||
s1.statsFetcher.inflight[string(s3.config.NodeID)] = struct{}{}
|
||||
defer delete(s1.statsFetcher.inflight, string(s3.config.NodeID))
|
||||
s1.statsFetcher.inflight[raft.ServerID(s3.config.NodeID)] = struct{}{}
|
||||
defer delete(s1.statsFetcher.inflight, raft.ServerID(s3.config.NodeID))
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
|
||||
defer cancel()
|
||||
stats := s1.statsFetcher.Fetch(ctx, s1.LANMembers())
|
||||
stats := s1.statsFetcher.Fetch(ctx, s1.autopilotServers())
|
||||
if len(stats) != 2 {
|
||||
r.Fatalf("bad: %#v", stats)
|
||||
}
|
||||
|
|
|
@ -4,7 +4,6 @@ import (
|
|||
"fmt"
|
||||
"strconv"
|
||||
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
)
|
||||
|
||||
|
@ -57,18 +56,18 @@ func (s *Status) Peers(args *structs.DCSpecificRequest, reply *[]string) error {
|
|||
}
|
||||
|
||||
// Used by Autopilot to query the raft stats of the local server.
|
||||
func (s *Status) RaftStats(args struct{}, reply *autopilot.ServerStats) error {
|
||||
func (s *Status) RaftStats(args struct{}, reply *structs.RaftStats) error {
|
||||
stats := s.server.raft.Stats()
|
||||
|
||||
var err error
|
||||
reply.LastContact = stats["last_contact"]
|
||||
reply.LastIndex, err = strconv.ParseUint(stats["last_log_index"], 10, 64)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error parsing server's last_log_index value: %s", err)
|
||||
return fmt.Errorf("error parsing server's last_log_index value: %w", err)
|
||||
}
|
||||
reply.LastTerm, err = strconv.ParseUint(stats["last_log_term"], 10, 64)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error parsing server's last_log_term value: %s", err)
|
||||
return fmt.Errorf("error parsing server's last_log_term value: %w", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
|
|
|
@ -108,6 +108,7 @@ func init() {
|
|||
registerEndpoint("/v1/operator/keyring", []string{"GET", "POST", "PUT", "DELETE"}, (*HTTPHandlers).OperatorKeyringEndpoint)
|
||||
registerEndpoint("/v1/operator/autopilot/configuration", []string{"GET", "PUT"}, (*HTTPHandlers).OperatorAutopilotConfiguration)
|
||||
registerEndpoint("/v1/operator/autopilot/health", []string{"GET"}, (*HTTPHandlers).OperatorServerHealth)
|
||||
registerEndpoint("/v1/operator/autopilot/state", []string{"GET"}, (*HTTPHandlers).OperatorAutopilotState)
|
||||
registerEndpoint("/v1/query", []string{"GET", "POST"}, (*HTTPHandlers).PreparedQueryGeneral)
|
||||
// specific prepared query endpoints have more complex rules for allowed methods, so
|
||||
// the prefix is registered with no methods.
|
||||
|
|
|
@ -905,6 +905,8 @@ func (l *State) updateSyncState() error {
|
|||
!reflect.DeepEqual(svcNode.TaggedAddresses, l.config.TaggedAddresses) ||
|
||||
!reflect.DeepEqual(svcNode.Meta, l.metadata) {
|
||||
l.nodeInfoInSync = false
|
||||
} else {
|
||||
|
||||
}
|
||||
|
||||
// Check which services need syncing
|
||||
|
|
|
@ -6,11 +6,11 @@ import (
|
|||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/api"
|
||||
multierror "github.com/hashicorp/go-multierror"
|
||||
"github.com/hashicorp/raft"
|
||||
autopilot "github.com/hashicorp/raft-autopilot"
|
||||
)
|
||||
|
||||
// OperatorRaftConfiguration is used to inspect the current Raft configuration.
|
||||
|
@ -192,7 +192,7 @@ func (s *HTTPHandlers) OperatorAutopilotConfiguration(resp http.ResponseWriter,
|
|||
return nil, nil
|
||||
}
|
||||
|
||||
var reply autopilot.Config
|
||||
var reply structs.AutopilotConfig
|
||||
if err := s.agent.RPC("Operator.AutopilotGetConfiguration", &args, &reply); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -222,7 +222,7 @@ func (s *HTTPHandlers) OperatorAutopilotConfiguration(resp http.ResponseWriter,
|
|||
return nil, BadRequestError{Reason: fmt.Sprintf("Error parsing autopilot config: %v", err)}
|
||||
}
|
||||
|
||||
args.Config = autopilot.Config{
|
||||
args.Config = structs.AutopilotConfig{
|
||||
CleanupDeadServers: conf.CleanupDeadServers,
|
||||
LastContactThreshold: conf.LastContactThreshold.Duration(),
|
||||
MaxTrailingLogs: conf.MaxTrailingLogs,
|
||||
|
@ -267,7 +267,7 @@ func (s *HTTPHandlers) OperatorServerHealth(resp http.ResponseWriter, req *http.
|
|||
return nil, nil
|
||||
}
|
||||
|
||||
var reply autopilot.OperatorHealthReply
|
||||
var reply structs.AutopilotHealthReply
|
||||
if err := s.agent.RPC("Operator.ServerHealth", &args, &reply); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -300,3 +300,66 @@ func (s *HTTPHandlers) OperatorServerHealth(resp http.ResponseWriter, req *http.
|
|||
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (s *HTTPHandlers) OperatorAutopilotState(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
|
||||
var args structs.DCSpecificRequest
|
||||
if done := s.parse(resp, req, &args.Datacenter, &args.QueryOptions); done {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
var reply autopilot.State
|
||||
if err := s.agent.RPC("Operator.AutopilotState", &args, &reply); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
out := autopilotToAPIState(&reply)
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func stringIDs(ids []raft.ServerID) []string {
|
||||
out := make([]string, len(ids))
|
||||
for i, id := range ids {
|
||||
out[i] = string(id)
|
||||
}
|
||||
return out
|
||||
}
|
||||
|
||||
func autopilotToAPIState(state *autopilot.State) *api.AutopilotState {
|
||||
out := &api.AutopilotState{
|
||||
Healthy: state.Healthy,
|
||||
FailureTolerance: state.FailureTolerance,
|
||||
Leader: string(state.Leader),
|
||||
Voters: stringIDs(state.Voters),
|
||||
Servers: make(map[string]api.AutopilotServer),
|
||||
}
|
||||
|
||||
for id, srv := range state.Servers {
|
||||
out.Servers[string(id)] = autopilotToAPIServer(srv)
|
||||
}
|
||||
|
||||
autopilotToAPIStateEnterprise(state, out)
|
||||
|
||||
return out
|
||||
}
|
||||
|
||||
func autopilotToAPIServer(srv *autopilot.ServerState) api.AutopilotServer {
|
||||
apiSrv := api.AutopilotServer{
|
||||
ID: string(srv.Server.ID),
|
||||
Name: srv.Server.Name,
|
||||
Address: string(srv.Server.Address),
|
||||
NodeStatus: string(srv.Server.NodeStatus),
|
||||
Version: srv.Server.Version,
|
||||
LastContact: api.NewReadableDuration(srv.Stats.LastContact),
|
||||
LastTerm: srv.Stats.LastTerm,
|
||||
LastIndex: srv.Stats.LastIndex,
|
||||
Healthy: srv.Health.Healthy,
|
||||
StableSince: srv.Health.StableSince,
|
||||
Status: api.AutopilotServerStatus(srv.State),
|
||||
Meta: srv.Server.Meta,
|
||||
NodeType: api.AutopilotServerType(srv.Server.NodeType),
|
||||
}
|
||||
|
||||
autopilotToAPIServerEnterprise(srv, &apiSrv)
|
||||
|
||||
return apiSrv
|
||||
}
|
||||
|
|
|
@ -0,0 +1,16 @@
|
|||
// +build !consulent
|
||||
|
||||
package agent
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/consul/api"
|
||||
autopilot "github.com/hashicorp/raft-autopilot"
|
||||
)
|
||||
|
||||
func autopilotToAPIServerEnterprise(_ *autopilot.ServerState, _ *api.AutopilotServer) {
|
||||
// noop in oss
|
||||
}
|
||||
|
||||
func autopilotToAPIStateEnterprise(_ *autopilot.State, _ *api.AutopilotState) {
|
||||
// noop in oss
|
||||
}
|
|
@ -7,11 +7,13 @@ import (
|
|||
"net/http/httptest"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/consul/testrpc"
|
||||
"github.com/hashicorp/raft"
|
||||
autopilot "github.com/hashicorp/raft-autopilot"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/hashicorp/consul/api"
|
||||
"github.com/hashicorp/consul/sdk/testutil/retry"
|
||||
|
@ -403,7 +405,7 @@ func TestOperator_AutopilotSetConfiguration(t *testing.T) {
|
|||
Datacenter: "dc1",
|
||||
}
|
||||
|
||||
var reply autopilot.Config
|
||||
var reply structs.AutopilotConfig
|
||||
if err := a.RPC("Operator.AutopilotGetConfiguration", &args, &reply); err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
@ -432,7 +434,7 @@ func TestOperator_AutopilotCASConfiguration(t *testing.T) {
|
|||
Datacenter: "dc1",
|
||||
}
|
||||
|
||||
var reply autopilot.Config
|
||||
var reply structs.AutopilotConfig
|
||||
if err := a.RPC("Operator.AutopilotGetConfiguration", &args, &reply); err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
@ -544,3 +546,178 @@ func TestOperator_ServerHealth_Unhealthy(t *testing.T) {
|
|||
}
|
||||
})
|
||||
}
|
||||
|
||||
func TestOperator_AutopilotState(t *testing.T) {
|
||||
a := NewTestAgent(t, "")
|
||||
defer a.Shutdown()
|
||||
|
||||
req, err := http.NewRequest("GET", "/v1/operator/autopilot/state", nil)
|
||||
require.NoError(t, err)
|
||||
retry.Run(t, func(r *retry.R) {
|
||||
resp := httptest.NewRecorder()
|
||||
obj, err := a.srv.OperatorAutopilotState(resp, req)
|
||||
require.NoError(r, err)
|
||||
require.Equal(r, 200, resp.Code)
|
||||
state, ok := obj.(*api.AutopilotState)
|
||||
require.True(r, ok)
|
||||
|
||||
srv, ok := state.Servers[string(a.config.NodeID)]
|
||||
require.True(r, ok)
|
||||
require.True(r, srv.Healthy)
|
||||
require.Equal(r, a.config.NodeName, srv.Name)
|
||||
|
||||
})
|
||||
}
|
||||
|
||||
func TestAutopilotStateToAPIConversion(t *testing.T) {
|
||||
var leaderID raft.ServerID = "79324811-9588-4311-b208-f272e38aaabf"
|
||||
var follower1ID raft.ServerID = "ef8aee9a-f9d6-4ec4-b383-aac956bdb80f"
|
||||
var follower2ID raft.ServerID = "ae84aefb-a303-4734-8739-5c102d4ee2d9"
|
||||
input := autopilot.State{
|
||||
Healthy: true,
|
||||
FailureTolerance: 1,
|
||||
Leader: leaderID,
|
||||
Voters: []raft.ServerID{
|
||||
leaderID,
|
||||
follower1ID,
|
||||
follower2ID,
|
||||
},
|
||||
Servers: map[raft.ServerID]*autopilot.ServerState{
|
||||
leaderID: {
|
||||
Server: autopilot.Server{
|
||||
ID: leaderID,
|
||||
Name: "node1",
|
||||
Address: "198.18.0.1:8300",
|
||||
NodeStatus: autopilot.NodeAlive,
|
||||
Version: "1.9.0",
|
||||
Meta: map[string]string{
|
||||
"foo": "bar",
|
||||
},
|
||||
NodeType: autopilot.NodeVoter,
|
||||
},
|
||||
State: autopilot.RaftLeader,
|
||||
Stats: autopilot.ServerStats{
|
||||
LastContact: 0,
|
||||
LastTerm: 3,
|
||||
LastIndex: 42,
|
||||
},
|
||||
Health: autopilot.ServerHealth{
|
||||
Healthy: true,
|
||||
StableSince: time.Date(2020, 11, 6, 14, 51, 0, 0, time.UTC),
|
||||
},
|
||||
},
|
||||
follower1ID: {
|
||||
Server: autopilot.Server{
|
||||
ID: follower1ID,
|
||||
Name: "node2",
|
||||
Address: "198.18.0.2:8300",
|
||||
NodeStatus: autopilot.NodeAlive,
|
||||
Version: "1.9.0",
|
||||
Meta: map[string]string{
|
||||
"bar": "baz",
|
||||
},
|
||||
NodeType: autopilot.NodeVoter,
|
||||
},
|
||||
State: autopilot.RaftVoter,
|
||||
Stats: autopilot.ServerStats{
|
||||
LastContact: time.Millisecond,
|
||||
LastTerm: 3,
|
||||
LastIndex: 41,
|
||||
},
|
||||
Health: autopilot.ServerHealth{
|
||||
Healthy: true,
|
||||
StableSince: time.Date(2020, 11, 6, 14, 52, 0, 0, time.UTC),
|
||||
},
|
||||
},
|
||||
follower2ID: {
|
||||
Server: autopilot.Server{
|
||||
ID: follower2ID,
|
||||
Name: "node3",
|
||||
Address: "198.18.0.3:8300",
|
||||
NodeStatus: autopilot.NodeAlive,
|
||||
Version: "1.9.0",
|
||||
Meta: map[string]string{
|
||||
"baz": "foo",
|
||||
},
|
||||
NodeType: autopilot.NodeVoter,
|
||||
},
|
||||
State: autopilot.RaftVoter,
|
||||
Stats: autopilot.ServerStats{
|
||||
LastContact: 2 * time.Millisecond,
|
||||
LastTerm: 3,
|
||||
LastIndex: 39,
|
||||
},
|
||||
Health: autopilot.ServerHealth{
|
||||
Healthy: true,
|
||||
StableSince: time.Date(2020, 11, 6, 14, 53, 0, 0, time.UTC),
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
expected := api.AutopilotState{
|
||||
Healthy: true,
|
||||
FailureTolerance: 1,
|
||||
Leader: string(leaderID),
|
||||
Voters: []string{
|
||||
string(leaderID),
|
||||
string(follower1ID),
|
||||
string(follower2ID),
|
||||
},
|
||||
Servers: map[string]api.AutopilotServer{
|
||||
string(leaderID): {
|
||||
ID: string(leaderID),
|
||||
Name: "node1",
|
||||
Address: "198.18.0.1:8300",
|
||||
NodeStatus: "alive",
|
||||
Version: "1.9.0",
|
||||
Meta: map[string]string{
|
||||
"foo": "bar",
|
||||
},
|
||||
NodeType: api.AutopilotTypeVoter,
|
||||
Status: api.AutopilotServerLeader,
|
||||
LastContact: api.NewReadableDuration(0),
|
||||
LastTerm: 3,
|
||||
LastIndex: 42,
|
||||
Healthy: true,
|
||||
StableSince: time.Date(2020, 11, 6, 14, 51, 0, 0, time.UTC),
|
||||
},
|
||||
string(follower1ID): {
|
||||
ID: string(follower1ID),
|
||||
Name: "node2",
|
||||
Address: "198.18.0.2:8300",
|
||||
NodeStatus: "alive",
|
||||
Version: "1.9.0",
|
||||
Meta: map[string]string{
|
||||
"bar": "baz",
|
||||
},
|
||||
NodeType: api.AutopilotTypeVoter,
|
||||
Status: api.AutopilotServerVoter,
|
||||
LastContact: api.NewReadableDuration(time.Millisecond),
|
||||
LastTerm: 3,
|
||||
LastIndex: 41,
|
||||
Healthy: true,
|
||||
StableSince: time.Date(2020, 11, 6, 14, 52, 0, 0, time.UTC),
|
||||
},
|
||||
string(follower2ID): {
|
||||
ID: string(follower2ID),
|
||||
Name: "node3",
|
||||
Address: "198.18.0.3:8300",
|
||||
NodeStatus: "alive",
|
||||
Version: "1.9.0",
|
||||
Meta: map[string]string{
|
||||
"baz": "foo",
|
||||
},
|
||||
NodeType: api.AutopilotTypeVoter,
|
||||
Status: api.AutopilotServerVoter,
|
||||
LastContact: api.NewReadableDuration(2 * time.Millisecond),
|
||||
LastTerm: 3,
|
||||
LastIndex: 39,
|
||||
Healthy: true,
|
||||
StableSince: time.Date(2020, 11, 6, 14, 53, 0, 0, time.UTC),
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
require.Equal(t, &expected, autopilotToAPIState(&input))
|
||||
}
|
||||
|
|
|
@ -1,13 +1,14 @@
|
|||
package autopilot
|
||||
package structs
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
autopilot "github.com/hashicorp/raft-autopilot"
|
||||
"github.com/hashicorp/serf/serf"
|
||||
)
|
||||
|
||||
// Config holds the Autopilot configuration for a cluster.
|
||||
type Config struct {
|
||||
// Autopilotconfig holds the Autopilot configuration for a cluster.
|
||||
type AutopilotConfig struct {
|
||||
// CleanupDeadServers controls whether to remove dead servers when a new
|
||||
// server is added to the Raft peers.
|
||||
CleanupDeadServers bool
|
||||
|
@ -47,8 +48,35 @@ type Config struct {
|
|||
ModifyIndex uint64
|
||||
}
|
||||
|
||||
func (c *AutopilotConfig) ToAutopilotLibraryConfig() *autopilot.Config {
|
||||
if c == nil {
|
||||
return nil
|
||||
}
|
||||
return &autopilot.Config{
|
||||
CleanupDeadServers: c.CleanupDeadServers,
|
||||
LastContactThreshold: c.LastContactThreshold,
|
||||
MaxTrailingLogs: c.MaxTrailingLogs,
|
||||
MinQuorum: c.MinQuorum,
|
||||
ServerStabilizationTime: c.ServerStabilizationTime,
|
||||
Ext: c.autopilotConfigExt(),
|
||||
}
|
||||
}
|
||||
|
||||
// AutopilotHealthReply is a representation of the overall health of the cluster
|
||||
type AutopilotHealthReply struct {
|
||||
// Healthy is true if all the servers in the cluster are healthy.
|
||||
Healthy bool
|
||||
|
||||
// FailureTolerance is the number of healthy servers that could be lost without
|
||||
// an outage occurring.
|
||||
FailureTolerance int
|
||||
|
||||
// Servers holds the health of each server.
|
||||
Servers []AutopilotServerHealth
|
||||
}
|
||||
|
||||
// ServerHealth is the health (from the leader's point of view) of a server.
|
||||
type ServerHealth struct {
|
||||
type AutopilotServerHealth struct {
|
||||
// ID is the raft ID of the server.
|
||||
ID string
|
||||
|
||||
|
@ -87,48 +115,8 @@ type ServerHealth struct {
|
|||
StableSince time.Time
|
||||
}
|
||||
|
||||
// IsHealthy determines whether this ServerHealth is considered healthy
|
||||
// based on the given Autopilot config
|
||||
func (h *ServerHealth) IsHealthy(lastTerm uint64, leaderLastIndex uint64, autopilotConf *Config) bool {
|
||||
if h.SerfStatus != serf.StatusAlive {
|
||||
return false
|
||||
}
|
||||
|
||||
if h.LastContact > autopilotConf.LastContactThreshold || h.LastContact < 0 {
|
||||
return false
|
||||
}
|
||||
|
||||
if h.LastTerm != lastTerm {
|
||||
return false
|
||||
}
|
||||
|
||||
if leaderLastIndex > autopilotConf.MaxTrailingLogs && h.LastIndex < leaderLastIndex-autopilotConf.MaxTrailingLogs {
|
||||
return false
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
// IsStable returns true if the ServerHealth shows a stable, passing state
|
||||
// according to the given AutopilotConfig
|
||||
func (h *ServerHealth) IsStable(now time.Time, conf *Config) bool {
|
||||
if h == nil {
|
||||
return false
|
||||
}
|
||||
|
||||
if !h.Healthy {
|
||||
return false
|
||||
}
|
||||
|
||||
if now.Sub(h.StableSince) < conf.ServerStabilizationTime {
|
||||
return false
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
// ServerStats holds miscellaneous Raft metrics for a server
|
||||
type ServerStats struct {
|
||||
// RaftStats holds miscellaneous Raft metrics for a server.
|
||||
type RaftStats struct {
|
||||
// LastContact is the time since this node's last contact with the leader.
|
||||
LastContact string
|
||||
|
||||
|
@ -139,24 +127,11 @@ type ServerStats struct {
|
|||
LastIndex uint64
|
||||
}
|
||||
|
||||
// OperatorHealthReply is a representation of the overall health of the cluster
|
||||
type OperatorHealthReply struct {
|
||||
// Healthy is true if all the servers in the cluster are healthy.
|
||||
Healthy bool
|
||||
|
||||
// FailureTolerance is the number of healthy servers that could be lost without
|
||||
// an outage occurring.
|
||||
FailureTolerance int
|
||||
|
||||
// Servers holds the health of each server.
|
||||
Servers []ServerHealth
|
||||
}
|
||||
|
||||
func (o *OperatorHealthReply) ServerHealth(id string) *ServerHealth {
|
||||
for _, health := range o.Servers {
|
||||
if health.ID == id {
|
||||
return &health
|
||||
}
|
||||
func (s *RaftStats) ToAutopilotServerStats() *autopilot.ServerStats {
|
||||
duration, _ := time.ParseDuration(s.LastContact)
|
||||
return &autopilot.ServerStats{
|
||||
LastContact: duration,
|
||||
LastTerm: s.LastTerm,
|
||||
LastIndex: s.LastIndex,
|
||||
}
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,7 @@
|
|||
// +build !consulent
|
||||
|
||||
package structs
|
||||
|
||||
func (c *AutopilotConfig) autopilotConfigExt() interface{} {
|
||||
return nil
|
||||
}
|
|
@ -3,7 +3,6 @@ package structs
|
|||
import (
|
||||
"net"
|
||||
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/raft"
|
||||
)
|
||||
|
||||
|
@ -72,7 +71,7 @@ type AutopilotSetConfigRequest struct {
|
|||
Datacenter string
|
||||
|
||||
// Config is the new Autopilot configuration to use.
|
||||
Config autopilot.Config
|
||||
Config AutopilotConfig
|
||||
|
||||
// CAS controls whether to use check-and-set semantics for this request.
|
||||
CAS bool
|
||||
|
|
|
@ -111,6 +111,122 @@ type OperatorHealthReply struct {
|
|||
Servers []ServerHealth
|
||||
}
|
||||
|
||||
type AutopilotState struct {
|
||||
Healthy bool
|
||||
FailureTolerance int
|
||||
OptimisitcFailureTolerance int
|
||||
|
||||
Servers map[string]AutopilotServer
|
||||
Leader string
|
||||
Voters []string
|
||||
ReadReplicas []string `json:",omitempty"`
|
||||
RedundancyZones map[string]AutopilotZone `json:",omitempty"`
|
||||
Upgrade *AutopilotUpgrade `json:",omitempty"`
|
||||
}
|
||||
|
||||
type AutopilotServer struct {
|
||||
ID string
|
||||
Name string
|
||||
Address string
|
||||
NodeStatus string
|
||||
Version string
|
||||
LastContact *ReadableDuration
|
||||
LastTerm uint64
|
||||
LastIndex uint64
|
||||
Healthy bool
|
||||
StableSince time.Time
|
||||
RedundancyZone string `json:",omitempty"`
|
||||
UpgradeVersion string `json:",omitempty"`
|
||||
ReadReplica bool `json:",omitempty"`
|
||||
Status AutopilotServerStatus
|
||||
Meta map[string]string
|
||||
NodeType AutopilotServerType
|
||||
}
|
||||
|
||||
type AutopilotServerStatus string
|
||||
|
||||
const (
|
||||
AutopilotServerNone AutopilotServerStatus = "none"
|
||||
AutopilotServerLeader AutopilotServerStatus = "leader"
|
||||
AutopilotServerVoter AutopilotServerStatus = "voter"
|
||||
AutopilotServerNonVoter AutopilotServerStatus = "non-voter"
|
||||
AutopilotServerStaging AutopilotServerStatus = "staging"
|
||||
)
|
||||
|
||||
type AutopilotServerType string
|
||||
|
||||
const (
|
||||
AutopilotTypeVoter AutopilotServerType = "voter"
|
||||
AutopilotTypeReadReplica AutopilotServerType = "read-replica"
|
||||
AutopilotTypeZoneVoter AutopilotServerType = "zone-voter"
|
||||
AutopilotTypeZoneExtraVoter AutopilotServerType = "zone-extra-voter"
|
||||
AutopilotTypeZoneStandby AutopilotServerType = "zone-standby"
|
||||
)
|
||||
|
||||
type AutopilotZone struct {
|
||||
Servers []string
|
||||
Voters []string
|
||||
FailureTolerance int
|
||||
}
|
||||
|
||||
type AutopilotZoneUpgradeVersions struct {
|
||||
TargetVersionVoters []string `json:",omitempty"`
|
||||
TargetVersionNonVoters []string `json:",omitempty"`
|
||||
OtherVersionVoters []string `json:",omitempty"`
|
||||
OtherVersionNonVoters []string `json:",omitempty"`
|
||||
}
|
||||
|
||||
type AutopilotUpgrade struct {
|
||||
Status AutopilotUpgradeStatus
|
||||
TargetVersion string `json:",omitempty"`
|
||||
TargetVersionVoters []string `json:",omitempty"`
|
||||
TargetVersionNonVoters []string `json:",omitempty"`
|
||||
TargetVersionReadReplicas []string `json:",omitempty"`
|
||||
OtherVersionVoters []string `json:",omitempty"`
|
||||
OtherVersionNonVoters []string `json:",omitempty"`
|
||||
OtherVersionReadReplicas []string `json:",omitempty"`
|
||||
RedundancyZones map[string]AutopilotZoneUpgradeVersions `json:",omitempty"`
|
||||
}
|
||||
|
||||
type AutopilotUpgradeStatus string
|
||||
|
||||
const (
|
||||
// AutopilotUpgradeIdle is the status when no upgrade is in progress.
|
||||
AutopilotUpgradeIdle AutopilotUpgradeStatus = "idle"
|
||||
|
||||
// AutopilotUpgradeAwaitNewVoters is the status when more servers of
|
||||
// the target version must be added in order to start the promotion
|
||||
// phase of the upgrade
|
||||
AutopilotUpgradeAwaitNewVoters AutopilotUpgradeStatus = "await-new-voters"
|
||||
|
||||
// AutopilotUpgradePromoting is the status when autopilot is promoting
|
||||
// servers of the target version.
|
||||
AutopilotUpgradePromoting AutopilotUpgradeStatus = "promoting"
|
||||
|
||||
// AutopilotUpgradeDemoting is the status when autopilot is demoting
|
||||
// servers not on the target version
|
||||
AutopilotUpgradeDemoting AutopilotUpgradeStatus = "demoting"
|
||||
|
||||
// AutopilotUpgradeLeaderTransfer is the status when autopilot is transferring
|
||||
// leadership from a server running an older version to a server
|
||||
// using the target version.
|
||||
AutopilotUpgradeLeaderTransfer AutopilotUpgradeStatus = "leader-transfer"
|
||||
|
||||
// AutopilotUpgradeAwaitNewServers is the status when autpilot has finished
|
||||
// transferring leadership and has demoted all the other versioned
|
||||
// servers but wants to indicate that more target version servers
|
||||
// are needed to replace all the existing other version servers.
|
||||
AutopilotUpgradeAwaitNewServers AutopilotUpgradeStatus = "await-new-servers"
|
||||
|
||||
// AutopilotUpgradeAwaitServerRemoval is the status when autopilot is waiting
|
||||
// for the servers on non-target versions to be removed
|
||||
AutopilotUpgradeAwaitServerRemoval AutopilotUpgradeStatus = "await-server-removal"
|
||||
|
||||
// AutopilotUpgradeDisabled is the status when automated ugprades are
|
||||
// disabled in the autopilot configuration
|
||||
AutopilotUpgradeDisabled AutopilotUpgradeStatus = "disabled"
|
||||
)
|
||||
|
||||
// ReadableDuration is a duration type that is serialized to JSON in human readable format.
|
||||
type ReadableDuration time.Duration
|
||||
|
||||
|
@ -230,3 +346,20 @@ func (op *Operator) AutopilotServerHealth(q *QueryOptions) (*OperatorHealthReply
|
|||
}
|
||||
return &out, nil
|
||||
}
|
||||
|
||||
func (op *Operator) AutopilotState(q *QueryOptions) (*AutopilotState, error) {
|
||||
r := op.c.newRequest("GET", "/v1/operator/autopilot/state")
|
||||
r.setQueryOptions(q)
|
||||
_, resp, err := requireOK(op.c.doRequest(r))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer resp.Body.Close()
|
||||
|
||||
var out AutopilotState
|
||||
if err := decodeBody(resp, &out); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &out, nil
|
||||
}
|
||||
|
|
|
@ -105,3 +105,21 @@ func TestAPI_OperatorAutopilotServerHealth(t *testing.T) {
|
|||
}
|
||||
})
|
||||
}
|
||||
|
||||
func TestAPI_OperatorAutopilotState(t *testing.T) {
|
||||
c, s := makeClient(t)
|
||||
defer s.Stop()
|
||||
|
||||
operator := c.Operator()
|
||||
retry.Run(t, func(r *retry.R) {
|
||||
out, err := operator.AutopilotState(nil)
|
||||
if err != nil {
|
||||
r.Fatalf("err: %v", err)
|
||||
}
|
||||
|
||||
srv, ok := out.Servers[s.Config.NodeID]
|
||||
if !ok || !srv.Healthy || srv.Name != s.Config.NodeName {
|
||||
r.Fatalf("bad: %v", out)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
|
|
@ -8,7 +8,6 @@ import (
|
|||
"github.com/hashicorp/consul/testrpc"
|
||||
|
||||
"github.com/hashicorp/consul/agent"
|
||||
"github.com/hashicorp/consul/agent/consul/autopilot"
|
||||
"github.com/hashicorp/consul/agent/structs"
|
||||
"github.com/mitchellh/cli"
|
||||
)
|
||||
|
@ -49,7 +48,7 @@ func TestOperatorAutopilotSetConfigCommand(t *testing.T) {
|
|||
req := structs.DCSpecificRequest{
|
||||
Datacenter: "dc1",
|
||||
}
|
||||
var reply autopilot.Config
|
||||
var reply structs.AutopilotConfig
|
||||
if err := a.RPC("Operator.AutopilotGetConfiguration", &req, &reply); err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
|
5
go.mod
5
go.mod
|
@ -36,7 +36,7 @@ require (
|
|||
github.com/hashicorp/go-cleanhttp v0.5.1
|
||||
github.com/hashicorp/go-connlimit v0.3.0
|
||||
github.com/hashicorp/go-discover v0.0.0-20200501174627-ad1e96bde088
|
||||
github.com/hashicorp/go-hclog v0.12.0
|
||||
github.com/hashicorp/go-hclog v0.14.1
|
||||
github.com/hashicorp/go-memdb v1.3.0
|
||||
github.com/hashicorp/go-msgpack v0.5.5
|
||||
github.com/hashicorp/go-multierror v1.1.0
|
||||
|
@ -53,6 +53,7 @@ require (
|
|||
github.com/hashicorp/memberlist v0.2.2
|
||||
github.com/hashicorp/net-rpc-msgpackrpc v0.0.0-20151116020338-a14192a58a69
|
||||
github.com/hashicorp/raft v1.2.0
|
||||
github.com/hashicorp/raft-autopilot v0.1.1
|
||||
github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea
|
||||
github.com/hashicorp/serf v0.9.5
|
||||
github.com/hashicorp/vault/api v1.0.5-0.20200717191844-f687267c8086
|
||||
|
@ -79,7 +80,7 @@ require (
|
|||
github.com/shirou/gopsutil/v3 v3.20.10
|
||||
github.com/stretchr/testify v1.6.1
|
||||
go.opencensus.io v0.22.0 // indirect
|
||||
go.uber.org/goleak v1.0.0
|
||||
go.uber.org/goleak v1.1.10
|
||||
golang.org/x/crypto v0.0.0-20200930160638-afb6bcd081ae
|
||||
golang.org/x/net v0.0.0-20200930145003-4acb6c075d10
|
||||
golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45
|
||||
|
|
10
go.sum
10
go.sum
|
@ -234,6 +234,8 @@ github.com/hashicorp/go-hclog v0.9.1/go.mod h1:5CU+agLiy3J7N7QjHK5d05KxGsuXiQLrj
|
|||
github.com/hashicorp/go-hclog v0.9.2/go.mod h1:5CU+agLiy3J7N7QjHK5d05KxGsuXiQLrjA0H7acj2lQ=
|
||||
github.com/hashicorp/go-hclog v0.12.0 h1:d4QkX8FRTYaKaCZBoXYY8zJX2BXjWxurN/GA2tkrmZM=
|
||||
github.com/hashicorp/go-hclog v0.12.0/go.mod h1:whpDNt7SSdeAju8AWKIWsul05p54N/39EeqMAyrmvFQ=
|
||||
github.com/hashicorp/go-hclog v0.14.1 h1:nQcJDQwIAGnmoUWp8ubocEX40cCml/17YkF6csQLReU=
|
||||
github.com/hashicorp/go-hclog v0.14.1/go.mod h1:whpDNt7SSdeAju8AWKIWsul05p54N/39EeqMAyrmvFQ=
|
||||
github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60=
|
||||
github.com/hashicorp/go-immutable-radix v1.3.0 h1:8exGP7ego3OmkfksihtSouGMZ+hQrhxx+FVELeXpVPE=
|
||||
github.com/hashicorp/go-immutable-radix v1.3.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60=
|
||||
|
@ -287,6 +289,8 @@ github.com/hashicorp/net-rpc-msgpackrpc v0.0.0-20151116020338-a14192a58a69/go.mo
|
|||
github.com/hashicorp/raft v1.1.1/go.mod h1:vPAJM8Asw6u8LxC3eJCUZmRP/E4QmUGE1R7g7k8sG/8=
|
||||
github.com/hashicorp/raft v1.2.0 h1:mHzHIrF0S91d3A7RPBvuqkgB4d/7oFJZyvf1Q4m7GA0=
|
||||
github.com/hashicorp/raft v1.2.0/go.mod h1:vPAJM8Asw6u8LxC3eJCUZmRP/E4QmUGE1R7g7k8sG/8=
|
||||
github.com/hashicorp/raft-autopilot v0.1.1 h1:f8Dv2y1Vq8ttuH2+oh5l87Paj/BINpMm5TBrMLx+qGQ=
|
||||
github.com/hashicorp/raft-autopilot v0.1.1/go.mod h1:HUBUSYtpQRVkgjvvoOgsZPvwe6b6FZJ1xXtaftRZvrA=
|
||||
github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea h1:xykPFhrBAS2J0VBzVa5e80b5ZtYuNQtgXjN40qBZlD4=
|
||||
github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea/go.mod h1:pNv7Wc3ycL6F5oOWn+tPGo2gWD4a5X+yp/ntwdKLjRk=
|
||||
github.com/hashicorp/serf v0.9.5 h1:EBWvyu9tcRszt3Bxp3KNssBMP1KuHWyO51lz9+786iM=
|
||||
|
@ -355,9 +359,7 @@ github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzp
|
|||
github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU=
|
||||
github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
|
||||
github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
|
||||
github.com/miekg/dns v1.1.26 h1:gPxPSwALAeHJSjarOs00QjVdV9QoBvc1D2ujQUr5BzU=
|
||||
github.com/miekg/dns v1.1.26/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKjuso=
|
||||
github.com/miekg/dns v1.1.27 h1:aEH/kqUzUxGJ/UHcEKdJY+ugH6WEzsEBBSPa8zuy1aM=
|
||||
github.com/miekg/dns v1.1.27/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM=
|
||||
github.com/miekg/dns v1.1.31 h1:sJFOl9BgwbYAWOGEwr61FU28pqsBNdpRBnhGXtO06Oo=
|
||||
github.com/miekg/dns v1.1.31/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM=
|
||||
|
@ -519,8 +521,8 @@ go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU=
|
|||
go.opencensus.io v0.22.0 h1:C9hSCOW830chIVkdja34wa6Ky+IzWllkUinR+BtRZd4=
|
||||
go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8=
|
||||
go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
|
||||
go.uber.org/goleak v1.0.0 h1:qsup4IcBdlmsnGfqyLl4Ntn3C2XCCuKAE7DwHpScyUo=
|
||||
go.uber.org/goleak v1.0.0/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A=
|
||||
go.uber.org/goleak v1.1.10 h1:z+mqJhf6ss6BSfSM671tgKyZBFPTTJM+HLxnhPC3wu0=
|
||||
go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A=
|
||||
go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
|
||||
go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
|
||||
golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
|
||||
|
|
|
@ -0,0 +1,71 @@
|
|||
package hclog
|
||||
|
||||
import (
|
||||
"regexp"
|
||||
"strings"
|
||||
)
|
||||
|
||||
// ExcludeByMessage provides a simple way to build a list of log messages that
|
||||
// can be queried and matched. This is meant to be used with the Exclude
|
||||
// option on Options to suppress log messages. This does not hold any mutexs
|
||||
// within itself, so normal usage would be to Add entries at setup and none after
|
||||
// Exclude is going to be called. Exclude is called with a mutex held within
|
||||
// the Logger, so that doesn't need to use a mutex. Example usage:
|
||||
//
|
||||
// f := new(ExcludeByMessage)
|
||||
// f.Add("Noisy log message text")
|
||||
// appLogger.Exclude = f.Exclude
|
||||
type ExcludeByMessage struct {
|
||||
messages map[string]struct{}
|
||||
}
|
||||
|
||||
// Add a message to be filtered. Do not call this after Exclude is to be called
|
||||
// due to concurrency issues.
|
||||
func (f *ExcludeByMessage) Add(msg string) {
|
||||
if f.messages == nil {
|
||||
f.messages = make(map[string]struct{})
|
||||
}
|
||||
|
||||
f.messages[msg] = struct{}{}
|
||||
}
|
||||
|
||||
// Return true if the given message should be included
|
||||
func (f *ExcludeByMessage) Exclude(level Level, msg string, args ...interface{}) bool {
|
||||
_, ok := f.messages[msg]
|
||||
return ok
|
||||
}
|
||||
|
||||
// ExcludeByPrefix is a simple type to match a message string that has a common prefix.
|
||||
type ExcludeByPrefix string
|
||||
|
||||
// Matches an message that starts with the prefix.
|
||||
func (p ExcludeByPrefix) Exclude(level Level, msg string, args ...interface{}) bool {
|
||||
return strings.HasPrefix(msg, string(p))
|
||||
}
|
||||
|
||||
// ExcludeByRegexp takes a regexp and uses it to match a log message string. If it matches
|
||||
// the log entry is excluded.
|
||||
type ExcludeByRegexp struct {
|
||||
Regexp *regexp.Regexp
|
||||
}
|
||||
|
||||
// Exclude the log message if the message string matches the regexp
|
||||
func (e ExcludeByRegexp) Exclude(level Level, msg string, args ...interface{}) bool {
|
||||
return e.Regexp.MatchString(msg)
|
||||
}
|
||||
|
||||
// ExcludeFuncs is a slice of functions that will called to see if a log entry
|
||||
// should be filtered or not. It stops calling functions once at least one returns
|
||||
// true.
|
||||
type ExcludeFuncs []func(level Level, msg string, args ...interface{}) bool
|
||||
|
||||
// Calls each function until one of them returns true
|
||||
func (ff ExcludeFuncs) Exclude(level Level, msg string, args ...interface{}) bool {
|
||||
for _, f := range ff {
|
||||
if f(level, msg, args...) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
|
@ -213,18 +213,34 @@ func (i *interceptLogger) DeregisterSink(sink SinkAdapter) {
|
|||
// Create a *log.Logger that will send it's data through this Logger. This
|
||||
// allows packages that expect to be using the standard library to log to
|
||||
// actually use this logger, which will also send to any registered sinks.
|
||||
func (l *interceptLogger) StandardLoggerIntercept(opts *StandardLoggerOptions) *log.Logger {
|
||||
func (i *interceptLogger) StandardLoggerIntercept(opts *StandardLoggerOptions) *log.Logger {
|
||||
if opts == nil {
|
||||
opts = &StandardLoggerOptions{}
|
||||
}
|
||||
|
||||
return log.New(l.StandardWriterIntercept(opts), "", 0)
|
||||
return log.New(i.StandardWriterIntercept(opts), "", 0)
|
||||
}
|
||||
|
||||
func (l *interceptLogger) StandardWriterIntercept(opts *StandardLoggerOptions) io.Writer {
|
||||
func (i *interceptLogger) StandardWriterIntercept(opts *StandardLoggerOptions) io.Writer {
|
||||
return &stdlogAdapter{
|
||||
log: l,
|
||||
log: i,
|
||||
inferLevels: opts.InferLevels,
|
||||
forceLevel: opts.ForceLevel,
|
||||
}
|
||||
}
|
||||
|
||||
func (i *interceptLogger) ResetOutput(opts *LoggerOptions) error {
|
||||
if or, ok := i.Logger.(OutputResettable); ok {
|
||||
return or.ResetOutput(opts)
|
||||
} else {
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (i *interceptLogger) ResetOutputWithFlush(opts *LoggerOptions, flushable Flushable) error {
|
||||
if or, ok := i.Logger.(OutputResettable); ok {
|
||||
return or.ResetOutputWithFlush(opts, flushable)
|
||||
} else {
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
|
|
@ -58,13 +58,15 @@ type intLogger struct {
|
|||
name string
|
||||
timeFormat string
|
||||
|
||||
// This is a pointer so that it's shared by any derived loggers, since
|
||||
// This is an interface so that it's shared by any derived loggers, since
|
||||
// those derived loggers share the bufio.Writer as well.
|
||||
mutex *sync.Mutex
|
||||
mutex Locker
|
||||
writer *writer
|
||||
level *int32
|
||||
|
||||
implied []interface{}
|
||||
|
||||
exclude func(level Level, msg string, args ...interface{}) bool
|
||||
}
|
||||
|
||||
// New returns a configured logger.
|
||||
|
@ -106,11 +108,14 @@ func newLogger(opts *LoggerOptions) *intLogger {
|
|||
mutex: mutex,
|
||||
writer: newWriter(output, opts.Color),
|
||||
level: new(int32),
|
||||
exclude: opts.Exclude,
|
||||
}
|
||||
|
||||
l.setColorization(opts)
|
||||
|
||||
if opts.TimeFormat != "" {
|
||||
if opts.DisableTime {
|
||||
l.timeFormat = ""
|
||||
} else if opts.TimeFormat != "" {
|
||||
l.timeFormat = opts.TimeFormat
|
||||
}
|
||||
|
||||
|
@ -131,6 +136,10 @@ func (l *intLogger) log(name string, level Level, msg string, args ...interface{
|
|||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
|
||||
if l.exclude != nil && l.exclude(level, msg, args...) {
|
||||
return
|
||||
}
|
||||
|
||||
if l.json {
|
||||
l.logJSON(t, name, level, msg, args...)
|
||||
} else {
|
||||
|
@ -169,12 +178,14 @@ func trimCallerPath(path string) string {
|
|||
return path[idx+1:]
|
||||
}
|
||||
|
||||
var logImplFile = regexp.MustCompile(`github.com/hashicorp/go-hclog/.+logger.go$`)
|
||||
var logImplFile = regexp.MustCompile(`.+intlogger.go|.+interceptlogger.go$`)
|
||||
|
||||
// Non-JSON logging format function
|
||||
func (l *intLogger) logPlain(t time.Time, name string, level Level, msg string, args ...interface{}) {
|
||||
l.writer.WriteString(t.Format(l.timeFormat))
|
||||
l.writer.WriteByte(' ')
|
||||
if len(l.timeFormat) > 0 {
|
||||
l.writer.WriteString(t.Format(l.timeFormat))
|
||||
l.writer.WriteByte(' ')
|
||||
}
|
||||
|
||||
s, ok := _levelToBracket[level]
|
||||
if ok {
|
||||
|
@ -260,6 +271,12 @@ func (l *intLogger) logPlain(t time.Time, name string, level Level, msg string,
|
|||
val = strconv.FormatUint(uint64(st), 10)
|
||||
case uint8:
|
||||
val = strconv.FormatUint(uint64(st), 10)
|
||||
case Hex:
|
||||
val = "0x" + strconv.FormatUint(uint64(st), 16)
|
||||
case Octal:
|
||||
val = "0" + strconv.FormatUint(uint64(st), 8)
|
||||
case Binary:
|
||||
val = "0b" + strconv.FormatUint(uint64(st), 2)
|
||||
case CapturedStacktrace:
|
||||
stacktrace = st
|
||||
continue FOR
|
||||
|
|
|
@ -5,7 +5,6 @@ import (
|
|||
"log"
|
||||
"os"
|
||||
"strings"
|
||||
"sync"
|
||||
)
|
||||
|
||||
var (
|
||||
|
@ -53,6 +52,18 @@ func Fmt(str string, args ...interface{}) Format {
|
|||
return append(Format{str}, args...)
|
||||
}
|
||||
|
||||
// A simple shortcut to format numbers in hex when displayed with the normal
|
||||
// text output. For example: L.Info("header value", Hex(17))
|
||||
type Hex int
|
||||
|
||||
// A simple shortcut to format numbers in octal when displayed with the normal
|
||||
// text output. For example: L.Info("perms", Octal(17))
|
||||
type Octal int
|
||||
|
||||
// A simple shortcut to format numbers in binary when displayed with the normal
|
||||
// text output. For example: L.Info("bits", Binary(17))
|
||||
type Binary int
|
||||
|
||||
// ColorOption expresses how the output should be colored, if at all.
|
||||
type ColorOption uint8
|
||||
|
||||
|
@ -90,6 +101,25 @@ func LevelFromString(levelStr string) Level {
|
|||
}
|
||||
}
|
||||
|
||||
func (l Level) String() string {
|
||||
switch l {
|
||||
case Trace:
|
||||
return "trace"
|
||||
case Debug:
|
||||
return "debug"
|
||||
case Info:
|
||||
return "info"
|
||||
case Warn:
|
||||
return "warn"
|
||||
case Error:
|
||||
return "error"
|
||||
case NoLevel:
|
||||
return "none"
|
||||
default:
|
||||
return "unknown"
|
||||
}
|
||||
}
|
||||
|
||||
// Logger describes the interface that must be implemeted by all loggers.
|
||||
type Logger interface {
|
||||
// Args are alternating key, val pairs
|
||||
|
@ -186,8 +216,10 @@ type LoggerOptions struct {
|
|||
// Where to write the logs to. Defaults to os.Stderr if nil
|
||||
Output io.Writer
|
||||
|
||||
// An optional mutex pointer in case Output is shared
|
||||
Mutex *sync.Mutex
|
||||
// An optional Locker in case Output is shared. This can be a sync.Mutex or
|
||||
// a NoopLocker if the caller wants control over output, e.g. for batching
|
||||
// log lines.
|
||||
Mutex Locker
|
||||
|
||||
// Control if the output should be in JSON.
|
||||
JSONFormat bool
|
||||
|
@ -198,9 +230,19 @@ type LoggerOptions struct {
|
|||
// The time format to use instead of the default
|
||||
TimeFormat string
|
||||
|
||||
// Control whether or not to display the time at all. This is required
|
||||
// because setting TimeFormat to empty assumes the default format.
|
||||
DisableTime bool
|
||||
|
||||
// Color the output. On Windows, colored logs are only avaiable for io.Writers that
|
||||
// are concretely instances of *os.File.
|
||||
Color ColorOption
|
||||
|
||||
// A function which is called with the log information and if it returns true the value
|
||||
// should not be logged.
|
||||
// This is useful when interacting with a system that you wish to suppress the log
|
||||
// message for (because it's too noisy, etc)
|
||||
Exclude func(level Level, msg string, args ...interface{}) bool
|
||||
}
|
||||
|
||||
// InterceptLogger describes the interface for using a logger
|
||||
|
@ -260,3 +302,26 @@ type OutputResettable interface {
|
|||
// given in opts will be used for the new output.
|
||||
ResetOutputWithFlush(opts *LoggerOptions, flushable Flushable) error
|
||||
}
|
||||
|
||||
// Locker is used for locking output. If not set when creating a logger, a
|
||||
// sync.Mutex will be used internally.
|
||||
type Locker interface {
|
||||
// Lock is called when the output is going to be changed or written to
|
||||
Lock()
|
||||
|
||||
// Unlock is called when the operation that called Lock() completes
|
||||
Unlock()
|
||||
}
|
||||
|
||||
// NoopLocker implements locker but does nothing. This is useful if the client
|
||||
// wants tight control over locking, in order to provide grouping of log
|
||||
// entries or other functionality.
|
||||
type NoopLocker struct{}
|
||||
|
||||
// Lock does nothing
|
||||
func (n NoopLocker) Lock() {}
|
||||
|
||||
// Unlock does nothing
|
||||
func (n NoopLocker) Unlock() {}
|
||||
|
||||
var _ Locker = (*NoopLocker)(nil)
|
||||
|
|
|
@ -2,6 +2,7 @@ package hclog
|
|||
|
||||
import (
|
||||
"bytes"
|
||||
"log"
|
||||
"strings"
|
||||
)
|
||||
|
||||
|
@ -72,3 +73,23 @@ func (s *stdlogAdapter) pickLevel(str string) (Level, string) {
|
|||
return Info, str
|
||||
}
|
||||
}
|
||||
|
||||
type logWriter struct {
|
||||
l *log.Logger
|
||||
}
|
||||
|
||||
func (l *logWriter) Write(b []byte) (int, error) {
|
||||
l.l.Println(string(bytes.TrimRight(b, " \n\t")))
|
||||
return len(b), nil
|
||||
}
|
||||
|
||||
// Takes a standard library logger and returns a Logger that will write to it
|
||||
func FromStandardLogger(l *log.Logger, opts *LoggerOptions) Logger {
|
||||
var dl LoggerOptions = *opts
|
||||
|
||||
// Use the time format that log.Logger uses
|
||||
dl.DisableTime = true
|
||||
dl.Output = &logWriter{l}
|
||||
|
||||
return New(&dl)
|
||||
}
|
||||
|
|
|
@ -0,0 +1,373 @@
|
|||
Mozilla Public License Version 2.0
|
||||
==================================
|
||||
|
||||
1. Definitions
|
||||
--------------
|
||||
|
||||
1.1. "Contributor"
|
||||
means each individual or legal entity that creates, contributes to
|
||||
the creation of, or owns Covered Software.
|
||||
|
||||
1.2. "Contributor Version"
|
||||
means the combination of the Contributions of others (if any) used
|
||||
by a Contributor and that particular Contributor's Contribution.
|
||||
|
||||
1.3. "Contribution"
|
||||
means Covered Software of a particular Contributor.
|
||||
|
||||
1.4. "Covered Software"
|
||||
means Source Code Form to which the initial Contributor has attached
|
||||
the notice in Exhibit A, the Executable Form of such Source Code
|
||||
Form, and Modifications of such Source Code Form, in each case
|
||||
including portions thereof.
|
||||
|
||||
1.5. "Incompatible With Secondary Licenses"
|
||||
means
|
||||
|
||||
(a) that the initial Contributor has attached the notice described
|
||||
in Exhibit B to the Covered Software; or
|
||||
|
||||
(b) that the Covered Software was made available under the terms of
|
||||
version 1.1 or earlier of the License, but not also under the
|
||||
terms of a Secondary License.
|
||||
|
||||
1.6. "Executable Form"
|
||||
means any form of the work other than Source Code Form.
|
||||
|
||||
1.7. "Larger Work"
|
||||
means a work that combines Covered Software with other material, in
|
||||
a separate file or files, that is not Covered Software.
|
||||
|
||||
1.8. "License"
|
||||
means this document.
|
||||
|
||||
1.9. "Licensable"
|
||||
means having the right to grant, to the maximum extent possible,
|
||||
whether at the time of the initial grant or subsequently, any and
|
||||
all of the rights conveyed by this License.
|
||||
|
||||
1.10. "Modifications"
|
||||
means any of the following:
|
||||
|
||||
(a) any file in Source Code Form that results from an addition to,
|
||||
deletion from, or modification of the contents of Covered
|
||||
Software; or
|
||||
|
||||
(b) any new file in Source Code Form that contains any Covered
|
||||
Software.
|
||||
|
||||
1.11. "Patent Claims" of a Contributor
|
||||
means any patent claim(s), including without limitation, method,
|
||||
process, and apparatus claims, in any patent Licensable by such
|
||||
Contributor that would be infringed, but for the grant of the
|
||||
License, by the making, using, selling, offering for sale, having
|
||||
made, import, or transfer of either its Contributions or its
|
||||
Contributor Version.
|
||||
|
||||
1.12. "Secondary License"
|
||||
means either the GNU General Public License, Version 2.0, the GNU
|
||||
Lesser General Public License, Version 2.1, the GNU Affero General
|
||||
Public License, Version 3.0, or any later versions of those
|
||||
licenses.
|
||||
|
||||
1.13. "Source Code Form"
|
||||
means the form of the work preferred for making modifications.
|
||||
|
||||
1.14. "You" (or "Your")
|
||||
means an individual or a legal entity exercising rights under this
|
||||
License. For legal entities, "You" includes any entity that
|
||||
controls, is controlled by, or is under common control with You. For
|
||||
purposes of this definition, "control" means (a) the power, direct
|
||||
or indirect, to cause the direction or management of such entity,
|
||||
whether by contract or otherwise, or (b) ownership of more than
|
||||
fifty percent (50%) of the outstanding shares or beneficial
|
||||
ownership of such entity.
|
||||
|
||||
2. License Grants and Conditions
|
||||
--------------------------------
|
||||
|
||||
2.1. Grants
|
||||
|
||||
Each Contributor hereby grants You a world-wide, royalty-free,
|
||||
non-exclusive license:
|
||||
|
||||
(a) under intellectual property rights (other than patent or trademark)
|
||||
Licensable by such Contributor to use, reproduce, make available,
|
||||
modify, display, perform, distribute, and otherwise exploit its
|
||||
Contributions, either on an unmodified basis, with Modifications, or
|
||||
as part of a Larger Work; and
|
||||
|
||||
(b) under Patent Claims of such Contributor to make, use, sell, offer
|
||||
for sale, have made, import, and otherwise transfer either its
|
||||
Contributions or its Contributor Version.
|
||||
|
||||
2.2. Effective Date
|
||||
|
||||
The licenses granted in Section 2.1 with respect to any Contribution
|
||||
become effective for each Contribution on the date the Contributor first
|
||||
distributes such Contribution.
|
||||
|
||||
2.3. Limitations on Grant Scope
|
||||
|
||||
The licenses granted in this Section 2 are the only rights granted under
|
||||
this License. No additional rights or licenses will be implied from the
|
||||
distribution or licensing of Covered Software under this License.
|
||||
Notwithstanding Section 2.1(b) above, no patent license is granted by a
|
||||
Contributor:
|
||||
|
||||
(a) for any code that a Contributor has removed from Covered Software;
|
||||
or
|
||||
|
||||
(b) for infringements caused by: (i) Your and any other third party's
|
||||
modifications of Covered Software, or (ii) the combination of its
|
||||
Contributions with other software (except as part of its Contributor
|
||||
Version); or
|
||||
|
||||
(c) under Patent Claims infringed by Covered Software in the absence of
|
||||
its Contributions.
|
||||
|
||||
This License does not grant any rights in the trademarks, service marks,
|
||||
or logos of any Contributor (except as may be necessary to comply with
|
||||
the notice requirements in Section 3.4).
|
||||
|
||||
2.4. Subsequent Licenses
|
||||
|
||||
No Contributor makes additional grants as a result of Your choice to
|
||||
distribute the Covered Software under a subsequent version of this
|
||||
License (see Section 10.2) or under the terms of a Secondary License (if
|
||||
permitted under the terms of Section 3.3).
|
||||
|
||||
2.5. Representation
|
||||
|
||||
Each Contributor represents that the Contributor believes its
|
||||
Contributions are its original creation(s) or it has sufficient rights
|
||||
to grant the rights to its Contributions conveyed by this License.
|
||||
|
||||
2.6. Fair Use
|
||||
|
||||
This License is not intended to limit any rights You have under
|
||||
applicable copyright doctrines of fair use, fair dealing, or other
|
||||
equivalents.
|
||||
|
||||
2.7. Conditions
|
||||
|
||||
Sections 3.1, 3.2, 3.3, and 3.4 are conditions of the licenses granted
|
||||
in Section 2.1.
|
||||
|
||||
3. Responsibilities
|
||||
-------------------
|
||||
|
||||
3.1. Distribution of Source Form
|
||||
|
||||
All distribution of Covered Software in Source Code Form, including any
|
||||
Modifications that You create or to which You contribute, must be under
|
||||
the terms of this License. You must inform recipients that the Source
|
||||
Code Form of the Covered Software is governed by the terms of this
|
||||
License, and how they can obtain a copy of this License. You may not
|
||||
attempt to alter or restrict the recipients' rights in the Source Code
|
||||
Form.
|
||||
|
||||
3.2. Distribution of Executable Form
|
||||
|
||||
If You distribute Covered Software in Executable Form then:
|
||||
|
||||
(a) such Covered Software must also be made available in Source Code
|
||||
Form, as described in Section 3.1, and You must inform recipients of
|
||||
the Executable Form how they can obtain a copy of such Source Code
|
||||
Form by reasonable means in a timely manner, at a charge no more
|
||||
than the cost of distribution to the recipient; and
|
||||
|
||||
(b) You may distribute such Executable Form under the terms of this
|
||||
License, or sublicense it under different terms, provided that the
|
||||
license for the Executable Form does not attempt to limit or alter
|
||||
the recipients' rights in the Source Code Form under this License.
|
||||
|
||||
3.3. Distribution of a Larger Work
|
||||
|
||||
You may create and distribute a Larger Work under terms of Your choice,
|
||||
provided that You also comply with the requirements of this License for
|
||||
the Covered Software. If the Larger Work is a combination of Covered
|
||||
Software with a work governed by one or more Secondary Licenses, and the
|
||||
Covered Software is not Incompatible With Secondary Licenses, this
|
||||
License permits You to additionally distribute such Covered Software
|
||||
under the terms of such Secondary License(s), so that the recipient of
|
||||
the Larger Work may, at their option, further distribute the Covered
|
||||
Software under the terms of either this License or such Secondary
|
||||
License(s).
|
||||
|
||||
3.4. Notices
|
||||
|
||||
You may not remove or alter the substance of any license notices
|
||||
(including copyright notices, patent notices, disclaimers of warranty,
|
||||
or limitations of liability) contained within the Source Code Form of
|
||||
the Covered Software, except that You may alter any license notices to
|
||||
the extent required to remedy known factual inaccuracies.
|
||||
|
||||
3.5. Application of Additional Terms
|
||||
|
||||
You may choose to offer, and to charge a fee for, warranty, support,
|
||||
indemnity or liability obligations to one or more recipients of Covered
|
||||
Software. However, You may do so only on Your own behalf, and not on
|
||||
behalf of any Contributor. You must make it absolutely clear that any
|
||||
such warranty, support, indemnity, or liability obligation is offered by
|
||||
You alone, and You hereby agree to indemnify every Contributor for any
|
||||
liability incurred by such Contributor as a result of warranty, support,
|
||||
indemnity or liability terms You offer. You may include additional
|
||||
disclaimers of warranty and limitations of liability specific to any
|
||||
jurisdiction.
|
||||
|
||||
4. Inability to Comply Due to Statute or Regulation
|
||||
---------------------------------------------------
|
||||
|
||||
If it is impossible for You to comply with any of the terms of this
|
||||
License with respect to some or all of the Covered Software due to
|
||||
statute, judicial order, or regulation then You must: (a) comply with
|
||||
the terms of this License to the maximum extent possible; and (b)
|
||||
describe the limitations and the code they affect. Such description must
|
||||
be placed in a text file included with all distributions of the Covered
|
||||
Software under this License. Except to the extent prohibited by statute
|
||||
or regulation, such description must be sufficiently detailed for a
|
||||
recipient of ordinary skill to be able to understand it.
|
||||
|
||||
5. Termination
|
||||
--------------
|
||||
|
||||
5.1. The rights granted under this License will terminate automatically
|
||||
if You fail to comply with any of its terms. However, if You become
|
||||
compliant, then the rights granted under this License from a particular
|
||||
Contributor are reinstated (a) provisionally, unless and until such
|
||||
Contributor explicitly and finally terminates Your grants, and (b) on an
|
||||
ongoing basis, if such Contributor fails to notify You of the
|
||||
non-compliance by some reasonable means prior to 60 days after You have
|
||||
come back into compliance. Moreover, Your grants from a particular
|
||||
Contributor are reinstated on an ongoing basis if such Contributor
|
||||
notifies You of the non-compliance by some reasonable means, this is the
|
||||
first time You have received notice of non-compliance with this License
|
||||
from such Contributor, and You become compliant prior to 30 days after
|
||||
Your receipt of the notice.
|
||||
|
||||
5.2. If You initiate litigation against any entity by asserting a patent
|
||||
infringement claim (excluding declaratory judgment actions,
|
||||
counter-claims, and cross-claims) alleging that a Contributor Version
|
||||
directly or indirectly infringes any patent, then the rights granted to
|
||||
You by any and all Contributors for the Covered Software under Section
|
||||
2.1 of this License shall terminate.
|
||||
|
||||
5.3. In the event of termination under Sections 5.1 or 5.2 above, all
|
||||
end user license agreements (excluding distributors and resellers) which
|
||||
have been validly granted by You or Your distributors under this License
|
||||
prior to termination shall survive termination.
|
||||
|
||||
************************************************************************
|
||||
* *
|
||||
* 6. Disclaimer of Warranty *
|
||||
* ------------------------- *
|
||||
* *
|
||||
* Covered Software is provided under this License on an "as is" *
|
||||
* basis, without warranty of any kind, either expressed, implied, or *
|
||||
* statutory, including, without limitation, warranties that the *
|
||||
* Covered Software is free of defects, merchantable, fit for a *
|
||||
* particular purpose or non-infringing. The entire risk as to the *
|
||||
* quality and performance of the Covered Software is with You. *
|
||||
* Should any Covered Software prove defective in any respect, You *
|
||||
* (not any Contributor) assume the cost of any necessary servicing, *
|
||||
* repair, or correction. This disclaimer of warranty constitutes an *
|
||||
* essential part of this License. No use of any Covered Software is *
|
||||
* authorized under this License except under this disclaimer. *
|
||||
* *
|
||||
************************************************************************
|
||||
|
||||
************************************************************************
|
||||
* *
|
||||
* 7. Limitation of Liability *
|
||||
* -------------------------- *
|
||||
* *
|
||||
* Under no circumstances and under no legal theory, whether tort *
|
||||
* (including negligence), contract, or otherwise, shall any *
|
||||
* Contributor, or anyone who distributes Covered Software as *
|
||||
* permitted above, be liable to You for any direct, indirect, *
|
||||
* special, incidental, or consequential damages of any character *
|
||||
* including, without limitation, damages for lost profits, loss of *
|
||||
* goodwill, work stoppage, computer failure or malfunction, or any *
|
||||
* and all other commercial damages or losses, even if such party *
|
||||
* shall have been informed of the possibility of such damages. This *
|
||||
* limitation of liability shall not apply to liability for death or *
|
||||
* personal injury resulting from such party's negligence to the *
|
||||
* extent applicable law prohibits such limitation. Some *
|
||||
* jurisdictions do not allow the exclusion or limitation of *
|
||||
* incidental or consequential damages, so this exclusion and *
|
||||
* limitation may not apply to You. *
|
||||
* *
|
||||
************************************************************************
|
||||
|
||||
8. Litigation
|
||||
-------------
|
||||
|
||||
Any litigation relating to this License may be brought only in the
|
||||
courts of a jurisdiction where the defendant maintains its principal
|
||||
place of business and such litigation shall be governed by laws of that
|
||||
jurisdiction, without reference to its conflict-of-law provisions.
|
||||
Nothing in this Section shall prevent a party's ability to bring
|
||||
cross-claims or counter-claims.
|
||||
|
||||
9. Miscellaneous
|
||||
----------------
|
||||
|
||||
This License represents the complete agreement concerning the subject
|
||||
matter hereof. If any provision of this License is held to be
|
||||
unenforceable, such provision shall be reformed only to the extent
|
||||
necessary to make it enforceable. Any law or regulation which provides
|
||||
that the language of a contract shall be construed against the drafter
|
||||
shall not be used to construe this License against a Contributor.
|
||||
|
||||
10. Versions of the License
|
||||
---------------------------
|
||||
|
||||
10.1. New Versions
|
||||
|
||||
Mozilla Foundation is the license steward. Except as provided in Section
|
||||
10.3, no one other than the license steward has the right to modify or
|
||||
publish new versions of this License. Each version will be given a
|
||||
distinguishing version number.
|
||||
|
||||
10.2. Effect of New Versions
|
||||
|
||||
You may distribute the Covered Software under the terms of the version
|
||||
of the License under which You originally received the Covered Software,
|
||||
or under the terms of any subsequent version published by the license
|
||||
steward.
|
||||
|
||||
10.3. Modified Versions
|
||||
|
||||
If you create software not governed by this License, and you want to
|
||||
create a new license for such software, you may create and use a
|
||||
modified version of this License if you rename the license and remove
|
||||
any references to the name of the license steward (except to note that
|
||||
such modified license differs from this License).
|
||||
|
||||
10.4. Distributing Source Code Form that is Incompatible With Secondary
|
||||
Licenses
|
||||
|
||||
If You choose to distribute Source Code Form that is Incompatible With
|
||||
Secondary Licenses under the terms of this version of the License, the
|
||||
notice described in Exhibit B of this License must be attached.
|
||||
|
||||
Exhibit A - Source Code Form License Notice
|
||||
-------------------------------------------
|
||||
|
||||
This Source Code Form is subject to the terms of the Mozilla Public
|
||||
License, v. 2.0. If a copy of the MPL was not distributed with this
|
||||
file, You can obtain one at http://mozilla.org/MPL/2.0/.
|
||||
|
||||
If it is not possible or desirable to put the notice in a particular
|
||||
file, then You may include the notice in a location (such as a LICENSE
|
||||
file in a relevant directory) where a recipient would be likely to look
|
||||
for such a notice.
|
||||
|
||||
You may add additional accurate notices of copyright ownership.
|
||||
|
||||
Exhibit B - "Incompatible With Secondary Licenses" Notice
|
||||
---------------------------------------------------------
|
||||
|
||||
This Source Code Form is "Incompatible With Secondary Licenses", as
|
||||
defined by the Mozilla Public License, v. 2.0.
|
|
@ -0,0 +1,2 @@
|
|||
# raft-autopilot
|
||||
Raft Autopilot
|
|
@ -0,0 +1,211 @@
|
|||
package autopilot
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
hclog "github.com/hashicorp/go-hclog"
|
||||
"github.com/hashicorp/raft"
|
||||
)
|
||||
|
||||
const (
|
||||
// These constants were take from what exists in Consul at the time of module extraction.
|
||||
|
||||
DefaultUpdateInterval = 2 * time.Second
|
||||
DefaultReconcileInterval = 10 * time.Second
|
||||
)
|
||||
|
||||
// Option is an option to be used when creating a new Autopilot instance
|
||||
type Option func(*Autopilot)
|
||||
|
||||
// WithUpdateInterval returns an Option to set the Autopilot instance's
|
||||
// update interval.
|
||||
func WithUpdateInterval(t time.Duration) Option {
|
||||
if t == 0 {
|
||||
t = DefaultUpdateInterval
|
||||
}
|
||||
return func(a *Autopilot) {
|
||||
a.updateInterval = t
|
||||
}
|
||||
}
|
||||
|
||||
// WithReconcileInterval returns an Option to set the Autopilot instance's
|
||||
// reconcile interval.
|
||||
func WithReconcileInterval(t time.Duration) Option {
|
||||
if t == 0 {
|
||||
t = DefaultReconcileInterval
|
||||
}
|
||||
return func(a *Autopilot) {
|
||||
a.reconcileInterval = t
|
||||
}
|
||||
}
|
||||
|
||||
// WithLogger returns an Option to set the Autopilot instance's logger
|
||||
func WithLogger(logger hclog.Logger) Option {
|
||||
if logger == nil {
|
||||
logger = hclog.Default()
|
||||
}
|
||||
|
||||
return func(a *Autopilot) {
|
||||
a.logger = logger.Named("autopilot")
|
||||
}
|
||||
}
|
||||
|
||||
// withTimeProvider returns an Option which overrides and Autopilot instance's
|
||||
// time provider with the given one. This should only be used in tests
|
||||
// as a means of making some time.Time values in an autopilot state deterministic.
|
||||
// For real uses the default runtimeTimeProvider should be used.
|
||||
func withTimeProvider(provider timeProvider) Option {
|
||||
return func(a *Autopilot) {
|
||||
a.time = provider
|
||||
}
|
||||
}
|
||||
|
||||
// WithPromoter returns an option to set the Promoter type that Autpilot will
|
||||
// use. When the option is not given the default StablePromoter from this package
|
||||
// will be used.
|
||||
func WithPromoter(promoter Promoter) Option {
|
||||
if promoter == nil {
|
||||
promoter = DefaultPromoter()
|
||||
}
|
||||
|
||||
return func(a *Autopilot) {
|
||||
a.promoter = promoter
|
||||
}
|
||||
}
|
||||
|
||||
// Autopilot is the type to manage a running Raft instance.
|
||||
//
|
||||
// Each Raft node in the cluster will have a corresponding Autopilot instance but
|
||||
// only 1 Autopilot instance should run at a time in the cluster. So when a node
|
||||
// gains Raft leadership the corresponding Autopilot instance should have it's
|
||||
// Start method called. Then if leadership is lost that node should call the
|
||||
// Stop method on the Autopilot instance.
|
||||
type Autopilot struct {
|
||||
logger hclog.Logger
|
||||
// delegate is used to get information about the system such as Raft server
|
||||
// states, known servers etc.
|
||||
delegate ApplicationIntegration
|
||||
// promoter is used to calculate promotions, demotions and leadership transfers
|
||||
// given a particular autopilot State. The interface also contains methods
|
||||
// for filling in parts of the autopilot state that the core module doesn't
|
||||
// control such as the Ext fields on the Server and State types.
|
||||
promoter Promoter
|
||||
// raft is an interface that implements all the parts of the Raft library interface
|
||||
// that we use. It is an interface to allow for mocking raft during testing.
|
||||
raft Raft
|
||||
// time is an interface with a single method for getting the current time - `Now`.
|
||||
// In some tests this will be the MockTimeProvider which allows tests to be more
|
||||
// deterministic but for running systems this should not be overrided from the
|
||||
// default which is the runtimeTimeProvider and is a small shim around calling
|
||||
// time.Now.
|
||||
time timeProvider
|
||||
|
||||
// reconcileInterval is how long between rounds of performing promotions, demotions
|
||||
// and leadership transfers.
|
||||
reconcileInterval time.Duration
|
||||
|
||||
// updateInterval is the time between the periodic state updates. These periodic
|
||||
// state updates take in known servers from the delegate, request Raft stats be
|
||||
// fetched and pull in other inputs such as the Raft configuration to create
|
||||
// an updated view of the Autopilot State.
|
||||
updateInterval time.Duration
|
||||
|
||||
// state is the structure that autopilot uses to make decisions about what to do.
|
||||
// This field should be considered immutable and no modifications to an existing
|
||||
// state should be made but instead a new state is created and set to this field
|
||||
// while holding the stateLock.
|
||||
state *State
|
||||
// stateLock is meant to only protect the state field. This just prevents
|
||||
// the periodic state update and consumers requesting the autopilot state from
|
||||
// racing.
|
||||
stateLock sync.RWMutex
|
||||
|
||||
// startTime is recorded so that we can make better determinations about server
|
||||
// stability during the initial period of time after autopilot first starts.
|
||||
// If autopilot has just started the default behavior to check if a server is
|
||||
// stable will not work as it will ensure the server has been healthy for
|
||||
// the configured server stabilization time. If that configure time is longer
|
||||
// than the amount of time autopilot has been running you can run into issues
|
||||
// with leadership flapping during some scenarios where a cluster is being
|
||||
// brought up.
|
||||
startTime time.Time
|
||||
|
||||
// running is a simple bool to indicate whether the go routines to actually
|
||||
// execute autopilot are currently running
|
||||
running bool
|
||||
|
||||
// removeDeadCh is used to trigger the running autopilot go routines to
|
||||
// find and remove any dead/failed servers
|
||||
removeDeadCh chan struct{}
|
||||
|
||||
// reconcileCh is used to trigger an immediate round of reconciliation.
|
||||
reconcileCh chan struct{}
|
||||
|
||||
// shutdown is a function that can be execute to shutdown a running
|
||||
// autopilot's go routines.
|
||||
shutdown context.CancelFunc
|
||||
// done is a chan that will be closed when the running autopilot go
|
||||
// routines have exited. Technically closing it is the very last
|
||||
// thing done in the go routine but at that point enough state has
|
||||
// been cleaned up that we would then allow it to be started
|
||||
// immediately afterward
|
||||
done chan struct{}
|
||||
|
||||
// runLock is meant to protect all of the fields regarding coordination
|
||||
// of whether the autopilot go routines are running and
|
||||
// starting/stopping them.
|
||||
runLock sync.Mutex
|
||||
}
|
||||
|
||||
// New will create a new Autopilot instance utilizing the given Raft and Delegate.
|
||||
// If the WithPromoter option is not provided the default StablePromoter will
|
||||
// be used.
|
||||
func New(raft Raft, delegate ApplicationIntegration, options ...Option) *Autopilot {
|
||||
a := &Autopilot{
|
||||
raft: raft,
|
||||
delegate: delegate,
|
||||
promoter: DefaultPromoter(),
|
||||
logger: hclog.Default().Named("autopilot"),
|
||||
// should this be buffered?
|
||||
removeDeadCh: make(chan struct{}, 1),
|
||||
reconcileInterval: DefaultReconcileInterval,
|
||||
updateInterval: DefaultUpdateInterval,
|
||||
time: &runtimeTimeProvider{},
|
||||
}
|
||||
|
||||
for _, opt := range options {
|
||||
opt(a)
|
||||
}
|
||||
|
||||
return a
|
||||
}
|
||||
|
||||
// RemoveDeadServers will trigger an immediate removal of dead/failed servers.
|
||||
func (a *Autopilot) RemoveDeadServers() {
|
||||
select {
|
||||
case a.removeDeadCh <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
// GetState retrieves the current autopilot State
|
||||
func (a *Autopilot) GetState() *State {
|
||||
a.stateLock.Lock()
|
||||
defer a.stateLock.Unlock()
|
||||
return a.state
|
||||
}
|
||||
|
||||
// GetServerHealth returns the latest ServerHealth for a given server.
|
||||
// The returned struct should not be modified or else it will im
|
||||
func (a *Autopilot) GetServerHealth(id raft.ServerID) *ServerHealth {
|
||||
state := a.GetState()
|
||||
|
||||
srv, ok := state.Servers[id]
|
||||
if ok {
|
||||
return &srv.Health
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,10 @@
|
|||
module github.com/hashicorp/raft-autopilot
|
||||
|
||||
go 1.14
|
||||
|
||||
require (
|
||||
github.com/hashicorp/go-hclog v0.14.1
|
||||
github.com/hashicorp/raft v1.2.0
|
||||
github.com/stretchr/testify v1.6.1
|
||||
go.uber.org/goleak v1.1.10
|
||||
)
|
|
@ -0,0 +1,86 @@
|
|||
github.com/DataDog/datadog-go v2.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ=
|
||||
github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878 h1:EFSB7Zo9Eg91v7MJPVsifUysc/wPdN+NOnVe6bWbdBM=
|
||||
github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878/go.mod h1:3AMJUQhVx52RsWOnlkpikZr01T/yAVN2gn0861vByNg=
|
||||
github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
|
||||
github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps=
|
||||
github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag=
|
||||
github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I=
|
||||
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
|
||||
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
|
||||
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
|
||||
github.com/fatih/color v1.7.0 h1:DkWD4oS2D8LGGgTQ6IvwJJXSL5Vp2ffcQg58nFV38Ys=
|
||||
github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4=
|
||||
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
|
||||
github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80=
|
||||
github.com/hashicorp/go-hclog v0.9.1 h1:9PZfAcVEvez4yhLH2TBU64/h/z4xlFI80cWXRrxuKuM=
|
||||
github.com/hashicorp/go-hclog v0.9.1/go.mod h1:5CU+agLiy3J7N7QjHK5d05KxGsuXiQLrjA0H7acj2lQ=
|
||||
github.com/hashicorp/go-hclog v0.14.1 h1:nQcJDQwIAGnmoUWp8ubocEX40cCml/17YkF6csQLReU=
|
||||
github.com/hashicorp/go-hclog v0.14.1/go.mod h1:whpDNt7SSdeAju8AWKIWsul05p54N/39EeqMAyrmvFQ=
|
||||
github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0=
|
||||
github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60=
|
||||
github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI=
|
||||
github.com/hashicorp/go-msgpack v0.5.5/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM=
|
||||
github.com/hashicorp/go-retryablehttp v0.5.3/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs=
|
||||
github.com/hashicorp/go-uuid v1.0.0 h1:RS8zrF7PhGwyNPOtxSClXXj9HA8feRnJzgnI1RJCSnM=
|
||||
github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
|
||||
github.com/hashicorp/golang-lru v0.5.0 h1:CL2msUPvZTLb5O648aiLNJw3hnBxN2+1Jq8rCOH9wdo=
|
||||
github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
|
||||
github.com/hashicorp/raft v1.2.0 h1:mHzHIrF0S91d3A7RPBvuqkgB4d/7oFJZyvf1Q4m7GA0=
|
||||
github.com/hashicorp/raft v1.2.0/go.mod h1:vPAJM8Asw6u8LxC3eJCUZmRP/E4QmUGE1R7g7k8sG/8=
|
||||
github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea/go.mod h1:pNv7Wc3ycL6F5oOWn+tPGo2gWD4a5X+yp/ntwdKLjRk=
|
||||
github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI=
|
||||
github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
|
||||
github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
|
||||
github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE=
|
||||
github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
|
||||
github.com/mattn/go-colorable v0.1.4 h1:snbPLB8fVfU9iwbbo30TPtbLRzwWu6aJS6Xh4eaaviA=
|
||||
github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE=
|
||||
github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
|
||||
github.com/mattn/go-isatty v0.0.10 h1:qxFzApOv4WsAL965uUPIsXzAKCZxN2p9UqdhFS4ZW10=
|
||||
github.com/mattn/go-isatty v0.0.10/go.mod h1:qgIWMr58cqv1PHHyhnkY9lrL7etaEgOFcMEpPG5Rm84=
|
||||
github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
|
||||
github.com/pascaldekloe/goe v0.1.0 h1:cBOtyMzM9HTpWjXfbbunk26uA6nG3a8n06Wieeh0MwY=
|
||||
github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
|
||||
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
|
||||
github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
|
||||
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
|
||||
github.com/prometheus/client_golang v0.9.2/go.mod h1:OsXs2jCmiKlQ1lTBmv21f2mNfw4xf/QclQDMrYNZzcM=
|
||||
github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
|
||||
github.com/prometheus/common v0.0.0-20181126121408-4724e9255275/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro=
|
||||
github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
|
||||
github.com/stretchr/objx v0.1.0 h1:4G4v2dO3VZwixGIRoQ5Lfboy6nUhCyYzaqnIAPPhYs4=
|
||||
github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
|
||||
github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
|
||||
github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q=
|
||||
github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
|
||||
github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
|
||||
github.com/stretchr/testify v1.6.1 h1:hDPOHmpOpP40lSULcqw7IrRb/u7w6RpDC9399XyoNd0=
|
||||
github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
|
||||
github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM=
|
||||
go.uber.org/goleak v1.1.10 h1:z+mqJhf6ss6BSfSM671tgKyZBFPTTJM+HLxnhPC3wu0=
|
||||
go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A=
|
||||
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
|
||||
golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs=
|
||||
golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
|
||||
golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||
golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
||||
golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
||||
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f h1:Bl/8QSvNqXvPGPGXa2z5xUTmV7VDcZyvRZ+QQXkXTZQ=
|
||||
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||
golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||
golang.org/x/sys v0.0.0-20190523142557-0e01d883c5c5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20191008105621-543471e840be h1:QAcqgptGM8IQBC9K/RC4o+O9YmqEm0diQn9QmZw/0mU=
|
||||
golang.org/x/sys v0.0.0-20191008105621-543471e840be/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
||||
golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||
golang.org/x/tools v0.0.0-20191108193012-7d206e10da11 h1:Yq9t9jnGoR+dBuitxdo9l6Q7xh/zOyNnYUtDKaQ3x0E=
|
||||
golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
|
||||
golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
|
||||
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY=
|
||||
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
|
||||
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
|
||||
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c h1:dUUwHk2QECo/6vqA44rthZ8ie2QXMNeKRTHCNY2nXvo=
|
||||
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
|
|
@ -0,0 +1,201 @@
|
|||
package autopilot
|
||||
|
||||
//
|
||||
// The methods in this file are all mainly to provide synchronous methods
|
||||
// for Raft operations that would normally return futures.
|
||||
//
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strconv"
|
||||
|
||||
"github.com/hashicorp/raft"
|
||||
)
|
||||
|
||||
func requiredQuorum(voters int) int {
|
||||
return (voters / 2) + 1
|
||||
}
|
||||
|
||||
// NumVoters is a helper for calculating the number of voting peers in the
|
||||
// current raft configuration. This function ignores any autopilot state
|
||||
// and will make the calculation based on a newly retrieved Raft configuration.
|
||||
func (a *Autopilot) NumVoters() (int, error) {
|
||||
cfg, err := a.getRaftConfiguration()
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
var numVoters int
|
||||
for _, server := range cfg.Servers {
|
||||
if server.Suffrage == raft.Voter {
|
||||
numVoters++
|
||||
}
|
||||
}
|
||||
|
||||
return numVoters, nil
|
||||
}
|
||||
|
||||
// AddServer is a helper for adding a new server to the raft configuration.
|
||||
// This may remove servers with duplicate addresses or ids first and after
|
||||
// its all done will trigger autopilot to remove dead servers if there
|
||||
// are any. Servers added by this method will start in a non-voting
|
||||
// state and later on autopilot will promote them to voting status
|
||||
// if desired by the configured promoter. If too many removals would
|
||||
// be required that would cause leadership loss then an error is returned
|
||||
// instead of performing any Raft configuration changes.
|
||||
func (a *Autopilot) AddServer(s *Server) error {
|
||||
cfg, err := a.getRaftConfiguration()
|
||||
if err != nil {
|
||||
a.logger.Error("failed to get raft configuration", "error", err)
|
||||
return err
|
||||
}
|
||||
|
||||
var existingVoter bool
|
||||
var voterRemovals []raft.ServerID
|
||||
var nonVoterRemovals []raft.ServerID
|
||||
var numVoters int
|
||||
for _, server := range cfg.Servers {
|
||||
if server.Suffrage == raft.Voter {
|
||||
numVoters++
|
||||
}
|
||||
|
||||
if server.Address == s.Address && server.ID == s.ID {
|
||||
// nothing to be done as the addr and ID both already match
|
||||
return nil
|
||||
} else if server.ID == s.ID {
|
||||
// special case for address updates only. In this case we should be
|
||||
// able to update the configuration without have to first remove the server
|
||||
if server.Suffrage == raft.Voter || server.Suffrage == raft.Staging {
|
||||
existingVoter = true
|
||||
}
|
||||
} else if server.Address == s.Address {
|
||||
if server.Suffrage == raft.Voter {
|
||||
voterRemovals = append(voterRemovals, server.ID)
|
||||
} else {
|
||||
nonVoterRemovals = append(nonVoterRemovals, server.ID)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
requiredVoters := requiredQuorum(numVoters)
|
||||
if len(voterRemovals) > numVoters-requiredVoters {
|
||||
return fmt.Errorf("Preventing server addition that would require removal of too many servers and cause cluster instability")
|
||||
}
|
||||
|
||||
for _, id := range voterRemovals {
|
||||
if err := a.removeServer(id); err != nil {
|
||||
return fmt.Errorf("error removing server %q with duplicate address %q: %w", id, s.Address, err)
|
||||
}
|
||||
a.logger.Info("removed server with duplicate address", "address", s.Address)
|
||||
}
|
||||
|
||||
for _, id := range nonVoterRemovals {
|
||||
if err := a.removeServer(id); err != nil {
|
||||
return fmt.Errorf("error removing server %q with duplicate address %q: %w", id, s.Address, err)
|
||||
}
|
||||
a.logger.Info("removed server with duplicate address", "address", s.Address)
|
||||
}
|
||||
|
||||
if existingVoter {
|
||||
if err := a.addVoter(s.ID, s.Address); err != nil {
|
||||
return err
|
||||
}
|
||||
} else {
|
||||
if err := a.addNonVoter(s.ID, s.Address); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Trigger a check to remove dead servers
|
||||
a.RemoveDeadServers()
|
||||
return nil
|
||||
}
|
||||
|
||||
// RemoveServer is a helper to remove a server from Raft if it
|
||||
// exists in the latest Raft configuration
|
||||
func (a *Autopilot) RemoveServer(id raft.ServerID) error {
|
||||
cfg, err := a.getRaftConfiguration()
|
||||
if err != nil {
|
||||
a.logger.Error("failed to get raft configuration", "error", err)
|
||||
return err
|
||||
}
|
||||
|
||||
// only remove servers currently in the configuration
|
||||
for _, server := range cfg.Servers {
|
||||
if server.ID == id {
|
||||
return a.removeServer(server.ID)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// addNonVoter is a wrapper around calling the AddNonVoter method on the Raft
|
||||
// interface object provided to Autopilot
|
||||
func (a *Autopilot) addNonVoter(id raft.ServerID, addr raft.ServerAddress) error {
|
||||
addFuture := a.raft.AddNonvoter(id, addr, 0, 0)
|
||||
if err := addFuture.Error(); err != nil {
|
||||
a.logger.Error("failed to add raft non-voting peer", "id", id, "address", addr, "error", err)
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// addVoter is a wrapper around calling the AddVoter method on the Raft
|
||||
// interface object provided to Autopilot
|
||||
func (a *Autopilot) addVoter(id raft.ServerID, addr raft.ServerAddress) error {
|
||||
addFuture := a.raft.AddVoter(id, addr, 0, 0)
|
||||
if err := addFuture.Error(); err != nil {
|
||||
a.logger.Error("failed to add raft voting peer", "id", id, "address", addr, "error", err)
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (a *Autopilot) demoteVoter(id raft.ServerID) error {
|
||||
removeFuture := a.raft.DemoteVoter(id, 0, 0)
|
||||
if err := removeFuture.Error(); err != nil {
|
||||
a.logger.Error("failed to demote raft peer", "id", id, "error", err)
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// removeServer is a wrapper around calling the RemoveServer method on the
|
||||
// Raft interface object provided to Autopilot
|
||||
func (a *Autopilot) removeServer(id raft.ServerID) error {
|
||||
a.logger.Debug("removing server by ID", "id", id)
|
||||
future := a.raft.RemoveServer(id, 0, 0)
|
||||
if err := future.Error(); err != nil {
|
||||
a.logger.Error("failed to remove raft server",
|
||||
"id", id,
|
||||
"error", err,
|
||||
)
|
||||
return err
|
||||
}
|
||||
a.logger.Info("removed server", "id", id)
|
||||
return nil
|
||||
}
|
||||
|
||||
// getRaftConfiguration a wrapper arond calling the GetConfiguration method
|
||||
// on the Raft interface object provided to Autopilot
|
||||
func (a *Autopilot) getRaftConfiguration() (*raft.Configuration, error) {
|
||||
configFuture := a.raft.GetConfiguration()
|
||||
if err := configFuture.Error(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
cfg := configFuture.Configuration()
|
||||
return &cfg, nil
|
||||
}
|
||||
|
||||
// lastTerm will retrieve the raft stats and then pull the last term value out of it
|
||||
func (a *Autopilot) lastTerm() (uint64, error) {
|
||||
return strconv.ParseUint(a.raft.Stats()["last_log_term"], 10, 64)
|
||||
}
|
||||
|
||||
// leadershipTransfer will transfer leadership to the server with the specified id and address
|
||||
func (a *Autopilot) leadershipTransfer(id raft.ServerID, address raft.ServerAddress) error {
|
||||
a.logger.Info("Transferring leadership to new server", "id", id, "address", address)
|
||||
future := a.raft.LeadershipTransferToServer(id, address)
|
||||
return future.Error()
|
||||
}
|
|
@ -0,0 +1,281 @@
|
|||
package autopilot
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sort"
|
||||
|
||||
"github.com/hashicorp/raft"
|
||||
)
|
||||
|
||||
// reconcile calculates and then applies promotions and demotions
|
||||
func (a *Autopilot) reconcile() error {
|
||||
conf := a.delegate.AutopilotConfig()
|
||||
if conf == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// grab the current state while locked
|
||||
a.stateLock.Lock()
|
||||
state := a.state
|
||||
a.stateLock.Unlock()
|
||||
|
||||
if state == nil || state.Leader == "" {
|
||||
return fmt.Errorf("Cannote reconcile Raft server voting rights without a valid autopilot state")
|
||||
}
|
||||
|
||||
// have the promoter calculate the required Raft changeset.
|
||||
changes := a.promoter.CalculatePromotionsAndDemotions(conf, state)
|
||||
|
||||
// apply the promotions, if we did apply any then stop here
|
||||
// as we do not want to apply the demotions at the same time
|
||||
// as a means of preventing cluster instability.
|
||||
if done, err := a.applyPromotions(state, changes); done {
|
||||
return err
|
||||
}
|
||||
|
||||
// apply the demotions, if we did apply any then stop here
|
||||
// as we do not want to transition leadership and do demotions
|
||||
// at the same time. This is a preventative measure to maintain
|
||||
// cluster stability.
|
||||
if done, err := a.applyDemotions(state, changes); done {
|
||||
return err
|
||||
}
|
||||
|
||||
// if no leadership transfer is desired then we can exit the method now.
|
||||
if changes.Leader == "" || changes.Leader == state.Leader {
|
||||
return nil
|
||||
}
|
||||
|
||||
// lookup the server we want to transfer leadership to
|
||||
srv, ok := state.Servers[changes.Leader]
|
||||
if !ok {
|
||||
return fmt.Errorf("cannot transfer leadership to an unknown server with ID %s", changes.Leader)
|
||||
}
|
||||
|
||||
// perform the leadership transfer
|
||||
return a.leadershipTransfer(changes.Leader, srv.Server.Address)
|
||||
}
|
||||
|
||||
// applyPromotions will apply all the promotions in the RaftChanges parameter.
|
||||
//
|
||||
// IDs in the change set will be ignored if:
|
||||
// * The server isn't tracked in the provided state
|
||||
// * The server already has voting rights
|
||||
// * The server is not healthy
|
||||
//
|
||||
// If any servers were promoted this function returns true for the bool value.
|
||||
func (a *Autopilot) applyPromotions(state *State, changes RaftChanges) (bool, error) {
|
||||
promoted := false
|
||||
for _, change := range changes.Promotions {
|
||||
srv, found := state.Servers[change]
|
||||
if !found {
|
||||
a.logger.Debug("Ignoring promotion of server as it is not in the autopilot state", "id", change)
|
||||
// this shouldn't be able to happen but is a nice safety measure against the
|
||||
// delegate doing something less than desirable
|
||||
continue
|
||||
}
|
||||
|
||||
if srv.HasVotingRights() {
|
||||
// There is no need to promote as this server is already a voter.
|
||||
// No logging is needed here as this could be a very common case
|
||||
// where the promoter just returns a lists of server ids that should
|
||||
// be voters and non-voters without caring about which ones currently
|
||||
// already are in that state.
|
||||
a.logger.Debug("Not promoting server that already has voting rights", "id", change)
|
||||
continue
|
||||
}
|
||||
|
||||
if !srv.Health.Healthy {
|
||||
// do not promote unhealthy servers
|
||||
a.logger.Debug("Ignoring promotion of unhealthy server", "id", change)
|
||||
continue
|
||||
}
|
||||
|
||||
a.logger.Info("Promoting server", "id", srv.Server.ID, "address", srv.Server.Address, "name", srv.Server.Name)
|
||||
|
||||
if err := a.addVoter(srv.Server.ID, srv.Server.Address); err != nil {
|
||||
return true, fmt.Errorf("failed promoting server %s: %v", srv.Server.ID, err)
|
||||
}
|
||||
|
||||
promoted = true
|
||||
}
|
||||
|
||||
// when we promoted anything we return true to indicate that the promotion/demotion applying
|
||||
// process is finished to prevent promotions and demotions in the same round. This is what
|
||||
// autopilot within Consul used to do so I am keeping the behavior the same for now.
|
||||
return promoted, nil
|
||||
}
|
||||
|
||||
// applyDemotions will apply all the demotions in the RaftChanges parameter.
|
||||
//
|
||||
// IDs in the change set will be ignored if:
|
||||
// * The server isn't tracked in the provided state
|
||||
// * The server does not have voting rights
|
||||
//
|
||||
// If any servers were demoted this function returns true for the bool value.
|
||||
func (a *Autopilot) applyDemotions(state *State, changes RaftChanges) (bool, error) {
|
||||
demoted := false
|
||||
for _, change := range changes.Demotions {
|
||||
srv, found := state.Servers[change]
|
||||
if !found {
|
||||
a.logger.Debug("Ignoring demotion of server as it is not in the autopilot state", "id", change)
|
||||
// this shouldn't be able to happen but is a nice safety measure against the
|
||||
// delegate doing something less than desirable
|
||||
continue
|
||||
}
|
||||
|
||||
if srv.State == RaftNonVoter {
|
||||
// There is no need to demote as this server is already a non-voter.
|
||||
// No logging is needed here as this could be a very common case
|
||||
// where the promoter just returns a lists of server ids that should
|
||||
// be voters and non-voters without caring about which ones currently
|
||||
// already are in that state.
|
||||
a.logger.Debug("Ignoring demotion of server that is already a non-voter", "id", change)
|
||||
continue
|
||||
}
|
||||
|
||||
a.logger.Info("Demoting server", "id", srv.Server.ID, "address", srv.Server.Address, "name", srv.Server.Name)
|
||||
|
||||
if err := a.demoteVoter(srv.Server.ID); err != nil {
|
||||
return true, fmt.Errorf("failed demoting server %s: %v", srv.Server.ID, err)
|
||||
}
|
||||
|
||||
demoted = true
|
||||
}
|
||||
|
||||
// similarly to applyPromotions here we want to stop the process and prevent leadership
|
||||
// transfer when any demotions took place. Basically we want to ensure the cluster is
|
||||
// stable before doing the transfer
|
||||
return demoted, nil
|
||||
}
|
||||
|
||||
// getFailedServers aggregates all of the information about servers that the consuming application believes are in
|
||||
// a failed/left state (indicated by the NodeStatus field on the Server type) as well as stale servers that are
|
||||
// in the raft configuration but not know to the consuming application. This function will do nothing with
|
||||
// that information and is purely to collect the data.
|
||||
func (a *Autopilot) getFailedServers() (*FailedServers, int, error) {
|
||||
staleRaftServers := make(map[raft.ServerID]raft.Server)
|
||||
raftConfig, err := a.getRaftConfiguration()
|
||||
if err != nil {
|
||||
return nil, 0, err
|
||||
}
|
||||
|
||||
// Populate a map of all the raft servers. We will
|
||||
// remove some later on from the map leaving us with
|
||||
// just the stale servers.
|
||||
var voters int
|
||||
for _, server := range raftConfig.Servers {
|
||||
staleRaftServers[server.ID] = server
|
||||
|
||||
if server.Suffrage == raft.Voter {
|
||||
voters++
|
||||
}
|
||||
}
|
||||
|
||||
var failed FailedServers
|
||||
for id, srv := range a.delegate.KnownServers() {
|
||||
raftSrv, found := staleRaftServers[id]
|
||||
if found {
|
||||
delete(staleRaftServers, id)
|
||||
}
|
||||
|
||||
if srv.NodeStatus != NodeAlive {
|
||||
if found && raftSrv.Suffrage == raft.Voter {
|
||||
failed.FailedVoters = append(failed.FailedVoters, srv)
|
||||
} else if found {
|
||||
failed.FailedNonVoters = append(failed.FailedNonVoters, srv)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for id, srv := range staleRaftServers {
|
||||
if srv.Suffrage == raft.Voter {
|
||||
failed.StaleVoters = append(failed.StaleVoters, id)
|
||||
} else {
|
||||
failed.StaleNonVoters = append(failed.StaleNonVoters, id)
|
||||
}
|
||||
}
|
||||
|
||||
sort.Slice(failed.StaleNonVoters, func(i, j int) bool {
|
||||
return failed.StaleNonVoters[i] < failed.StaleNonVoters[j]
|
||||
})
|
||||
sort.Slice(failed.StaleVoters, func(i, j int) bool {
|
||||
return failed.StaleVoters[i] < failed.StaleVoters[j]
|
||||
})
|
||||
sort.Slice(failed.FailedNonVoters, func(i, j int) bool {
|
||||
return failed.FailedNonVoters[i].ID < failed.FailedNonVoters[j].ID
|
||||
})
|
||||
sort.Slice(failed.FailedVoters, func(i, j int) bool {
|
||||
return failed.FailedVoters[i].ID < failed.FailedVoters[j].ID
|
||||
})
|
||||
|
||||
return &failed, voters, nil
|
||||
}
|
||||
|
||||
// pruneDeadServers will find stale raft servers and failed servers as indicated by the consuming application
|
||||
// and remove them. For stale raft servers this means removing them from the Raft configuration. For failed
|
||||
// servers this means issuing RemoveFailedNode calls to the delegate. All stale/failed non-voters will be
|
||||
// removed first. Then stale voters and finally failed servers. For servers with voting rights we will
|
||||
// cap the number removed so that we do not remove too many at a time and do not remove nodes to the
|
||||
// point where the number of voters would be below the MinQuorum value from the autopilot config.
|
||||
// Additionally the delegate will be consulted to determine if all of the removals should be done and
|
||||
// can filter the failed servers listings if need be.
|
||||
func (a *Autopilot) pruneDeadServers() error {
|
||||
conf := a.delegate.AutopilotConfig()
|
||||
if conf == nil || !conf.CleanupDeadServers {
|
||||
return nil
|
||||
}
|
||||
|
||||
state := a.GetState()
|
||||
|
||||
failed, voters, err := a.getFailedServers()
|
||||
if err != nil || failed == nil {
|
||||
return err
|
||||
}
|
||||
|
||||
failed = a.promoter.FilterFailedServerRemovals(conf, state, failed)
|
||||
|
||||
// remove failed non voting servers
|
||||
for _, srv := range failed.FailedNonVoters {
|
||||
a.logger.Info("Attempting removal of failed server node", "id", srv.ID, "name", srv.Name, "address", srv.Address)
|
||||
a.delegate.RemoveFailedServer(srv)
|
||||
}
|
||||
|
||||
// remove stale non voters
|
||||
for _, id := range failed.StaleNonVoters {
|
||||
a.logger.Debug("removing stale raft server from configuration", "id", id)
|
||||
if err := a.removeServer(id); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
maxRemoval := (voters - 1) / 2
|
||||
|
||||
for _, id := range failed.StaleVoters {
|
||||
if voters-1 < int(conf.MinQuorum) {
|
||||
a.logger.Debug("will not remove server as it would leave less voters than the minimum number allowed", "id", id, "min", conf.MinQuorum)
|
||||
} else if maxRemoval < 1 {
|
||||
a.logger.Debug("will not remove server as removal of a majority or servers is not safe", "id", id)
|
||||
} else if err := a.removeServer(id); err != nil {
|
||||
return err
|
||||
} else {
|
||||
maxRemoval--
|
||||
voters--
|
||||
}
|
||||
}
|
||||
|
||||
for _, srv := range failed.FailedVoters {
|
||||
if voters-1 < int(conf.MinQuorum) {
|
||||
a.logger.Debug("will not remove server as it would leave less voters than the minimum number allowed", "id", srv.ID, "min", conf.MinQuorum)
|
||||
} else if maxRemoval < 1 {
|
||||
a.logger.Debug("will not remove server as its removal would be unsafe due to affectingas removal of a majority or servers is not safe", "id", srv.ID)
|
||||
} else {
|
||||
a.logger.Info("Attempting removal of failed server node", "id", srv.ID, "name", srv.Name, "address", srv.Address)
|
||||
a.delegate.RemoveFailedServer(srv)
|
||||
maxRemoval--
|
||||
voters--
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,134 @@
|
|||
package autopilot
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
)
|
||||
|
||||
// Start will launch the go routines in the background to perform Autopilot.
|
||||
// When the context passed in is cancelled or the Stop method is called
|
||||
// then these routines will exit.
|
||||
func (a *Autopilot) Start(ctx context.Context) {
|
||||
a.runLock.Lock()
|
||||
defer a.runLock.Unlock()
|
||||
|
||||
// already running so there is nothing to do
|
||||
if a.running {
|
||||
return
|
||||
}
|
||||
|
||||
ctx, shutdown := context.WithCancel(ctx)
|
||||
a.shutdown = shutdown
|
||||
a.startTime = a.time.Now()
|
||||
a.done = make(chan struct{})
|
||||
|
||||
// While a go routine executed by a.run below will periodically
|
||||
// update the state, we want to go ahead and force updating it now
|
||||
// so that during a leadership transfer we don't report an empty
|
||||
// autopilot state. We put a pretty small timeout on this though
|
||||
// so as to prevent leader establishment from taking too long
|
||||
updateCtx, updateCancel := context.WithTimeout(ctx, time.Second)
|
||||
defer updateCancel()
|
||||
a.updateState(updateCtx)
|
||||
|
||||
go a.run(ctx)
|
||||
a.running = true
|
||||
}
|
||||
|
||||
// Stop will terminate the go routines being executed to perform autopilot.
|
||||
func (a *Autopilot) Stop() <-chan struct{} {
|
||||
a.runLock.Lock()
|
||||
defer a.runLock.Unlock()
|
||||
|
||||
// Nothing to do
|
||||
if !a.running {
|
||||
done := make(chan struct{})
|
||||
close(done)
|
||||
return done
|
||||
}
|
||||
|
||||
a.shutdown()
|
||||
return a.done
|
||||
}
|
||||
|
||||
func (a *Autopilot) run(ctx context.Context) {
|
||||
a.logger.Debug("autopilot is now running")
|
||||
// autopilot needs to do 3 things
|
||||
//
|
||||
// 1. periodically update the cluster state
|
||||
// 2. periodically check for and perform promotions and demotions
|
||||
// 3. Respond to servers leaving and prune dead servers
|
||||
//
|
||||
// We could attempt to do all of this in a single go routine except that
|
||||
// updating the cluster health could potentially take long enough to impact
|
||||
// the periodicity of the promotions and demotions performed by task 2/3.
|
||||
// So instead this go routine will spawn a second go routine to manage
|
||||
// updating the cluster health in the background. This go routine is still
|
||||
// in control of the overall running status and will not exit until the
|
||||
// child go routine has exited.
|
||||
|
||||
// child go routine for cluster health updating
|
||||
stateUpdaterDone := make(chan struct{})
|
||||
go a.runStateUpdater(ctx, stateUpdaterDone)
|
||||
|
||||
// cleanup for once we are stopped
|
||||
defer func() {
|
||||
// block waiting for our child go routine to also finish
|
||||
<-stateUpdaterDone
|
||||
|
||||
a.logger.Debug("autopilot is now stopped")
|
||||
|
||||
a.runLock.Lock()
|
||||
a.shutdown = nil
|
||||
a.running = false
|
||||
// this should be the final cleanup task as it is what notifies the rest
|
||||
// of the world that we are now done
|
||||
close(a.done)
|
||||
a.done = nil
|
||||
a.runLock.Unlock()
|
||||
}()
|
||||
|
||||
reconcileTicker := time.NewTicker(a.reconcileInterval)
|
||||
defer reconcileTicker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-reconcileTicker.C:
|
||||
if err := a.reconcile(); err != nil {
|
||||
a.logger.Error("Failed to reconcile current state with the desired state")
|
||||
}
|
||||
|
||||
if err := a.pruneDeadServers(); err != nil {
|
||||
a.logger.Error("Failed to prune dead servers", "error", err)
|
||||
}
|
||||
case <-a.removeDeadCh:
|
||||
if err := a.pruneDeadServers(); err != nil {
|
||||
a.logger.Error("Failed to prune dead servers", "error", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// runStateUpdated will periodically update the autopilot state until the context
|
||||
// passed in is cancelled. When finished the provide done chan will be closed.
|
||||
func (a *Autopilot) runStateUpdater(ctx context.Context, done chan struct{}) {
|
||||
a.logger.Debug("state update routine is now running")
|
||||
defer func() {
|
||||
a.logger.Debug("state update routine is now stopped")
|
||||
close(done)
|
||||
}()
|
||||
|
||||
ticker := time.NewTicker(a.updateInterval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
a.updateState(ctx)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
package autopilot
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/raft"
|
||||
)
|
||||
|
||||
func DefaultPromoter() Promoter {
|
||||
return new(StablePromoter)
|
||||
}
|
||||
|
||||
type StablePromoter struct{}
|
||||
|
||||
func (_ *StablePromoter) GetServerExt(_ *Config, srv *ServerState) interface{} {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (_ *StablePromoter) GetStateExt(_ *Config, _ *State) interface{} {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (_ *StablePromoter) GetNodeTypes(_ *Config, s *State) map[raft.ServerID]NodeType {
|
||||
types := make(map[raft.ServerID]NodeType)
|
||||
for id := range s.Servers {
|
||||
// this basic implementation has all nodes be of the "voter" type regardless of
|
||||
// any other settings. That means that in a healthy state all nodes in the cluster
|
||||
// will be a voter.
|
||||
types[id] = NodeVoter
|
||||
}
|
||||
return types
|
||||
}
|
||||
|
||||
func (_ *StablePromoter) FilterFailedServerRemovals(_ *Config, _ *State, failed *FailedServers) *FailedServers {
|
||||
return failed
|
||||
}
|
||||
|
||||
// CalculatePromotionsAndDemotions will return a list of all promotions and demotions to be done as well as the server id of
|
||||
// the desired leader. This particular interface implementation maintains a stable leader and will promote healthy servers
|
||||
// to voting status. It will never change the leader ID nor will it perform demotions.
|
||||
func (_ *StablePromoter) CalculatePromotionsAndDemotions(c *Config, s *State) RaftChanges {
|
||||
var changes RaftChanges
|
||||
|
||||
now := time.Now()
|
||||
minStableDuration := s.ServerStabilizationTime(c)
|
||||
for id, server := range s.Servers {
|
||||
// ignore staging state as they are not ready yet
|
||||
if server.State == RaftNonVoter && server.Health.IsStable(now, minStableDuration) {
|
||||
changes.Promotions = append(changes.Promotions, id)
|
||||
}
|
||||
}
|
||||
|
||||
return changes
|
||||
}
|
|
@ -0,0 +1,414 @@
|
|||
package autopilot
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sort"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/raft"
|
||||
)
|
||||
|
||||
// aliveServers will filter the input map of servers and output one with all of the
|
||||
// servers in a Left state removed.
|
||||
func aliveServers(servers map[raft.ServerID]*Server) map[raft.ServerID]*Server {
|
||||
serverMap := make(map[raft.ServerID]*Server)
|
||||
for _, server := range servers {
|
||||
if server.NodeStatus == NodeLeft {
|
||||
continue
|
||||
}
|
||||
|
||||
serverMap[server.ID] = server
|
||||
}
|
||||
|
||||
return serverMap
|
||||
}
|
||||
|
||||
// nextStateInputs is the collection of values that can influence
|
||||
// creation of the next State.
|
||||
type nextStateInputs struct {
|
||||
Now time.Time
|
||||
StartTime time.Time
|
||||
Config *Config
|
||||
State *State
|
||||
RaftConfig *raft.Configuration
|
||||
KnownServers map[raft.ServerID]*Server
|
||||
AliveServers map[raft.ServerID]*Server
|
||||
LatestIndex uint64
|
||||
LastTerm uint64
|
||||
FetchedStats map[raft.ServerID]*ServerStats
|
||||
LeaderID raft.ServerID
|
||||
}
|
||||
|
||||
// gatherNextStateInputs gathers all the information that would be used to
|
||||
// create the new updated state from.
|
||||
//
|
||||
// - Time Providers current time.
|
||||
// - Autopilot Config (needed to determine if the stats should indicate unhealthiness)
|
||||
// - Current state
|
||||
// - Raft Configuration
|
||||
// - Known Servers
|
||||
// - Latest raft index (gatered right before the remote server stats so that they should
|
||||
// be from about the same point in time)
|
||||
// - Stats for all non-left servers
|
||||
func (a *Autopilot) gatherNextStateInputs(ctx context.Context) (*nextStateInputs, error) {
|
||||
// we are going to hold this lock for the entire function. In theory nothing should
|
||||
// modify the state on any other go routine so this really shouldn't block anything
|
||||
// else. However we want to ensure that the inputs are as consistent as possible.
|
||||
a.stateLock.RLock()
|
||||
defer a.stateLock.RUnlock()
|
||||
|
||||
// there are a lot of inputs to computing the next state so they get put into a
|
||||
// struct so that we don't have to return 8 values.
|
||||
inputs := &nextStateInputs{
|
||||
Now: a.time.Now(),
|
||||
StartTime: a.startTime,
|
||||
}
|
||||
|
||||
// grab the latest autopilot configuration
|
||||
config := a.delegate.AutopilotConfig()
|
||||
if config == nil {
|
||||
return nil, fmt.Errorf("delegate did not return an Autopilot configuration")
|
||||
}
|
||||
inputs.Config = config
|
||||
|
||||
// retrieve the current state
|
||||
inputs.State = a.state
|
||||
if inputs.State == nil {
|
||||
inputs.State = &State{}
|
||||
}
|
||||
|
||||
// retrieve the raft configuration
|
||||
raftConfig, err := a.getRaftConfiguration()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to get the Raft configuration: %w", err)
|
||||
}
|
||||
inputs.RaftConfig = raftConfig
|
||||
|
||||
leader := a.raft.Leader()
|
||||
for _, s := range inputs.RaftConfig.Servers {
|
||||
if s.Address == leader {
|
||||
inputs.LeaderID = s.ID
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if inputs.LeaderID == "" {
|
||||
return nil, fmt.Errorf("cannot detect the current leader server id from its address: %s", leader)
|
||||
}
|
||||
|
||||
// get the latest Raft index - this should be kept close to the call to
|
||||
// fetch the statistics so that the index values are as close in time as
|
||||
// possible to make the best decision regarding an individual servers
|
||||
// healthiness.
|
||||
inputs.LatestIndex = a.raft.LastIndex()
|
||||
|
||||
term, err := a.lastTerm()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to determine the last Raft term: %w", err)
|
||||
}
|
||||
inputs.LastTerm = term
|
||||
|
||||
// getting the raft configuration could block for a while so now is a good
|
||||
// time to check for context cancellation
|
||||
if ctx.Err() != nil {
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
|
||||
// get the known servers which may include left/failed ones
|
||||
inputs.KnownServers = a.delegate.KnownServers()
|
||||
|
||||
// in most cases getting the known servers should be quick but as we cannot
|
||||
// account for every potential delegate and prevent them from making
|
||||
// blocking network requests we should probably check the context again.
|
||||
if ctx.Err() != nil {
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
|
||||
// filter the known servers to have a map of just the alive servers
|
||||
inputs.AliveServers = aliveServers(inputs.KnownServers)
|
||||
|
||||
// we only allow the fetch to take place for up to half the health interval
|
||||
// the next health interval will attempt to fetch the stats again but if
|
||||
// we do not see responses within this time then we can assume they are
|
||||
// unhealthy
|
||||
d := inputs.Now.Add(a.updateInterval / 2)
|
||||
fetchCtx, cancel := context.WithDeadline(ctx, d)
|
||||
defer cancel()
|
||||
|
||||
inputs.FetchedStats = a.delegate.FetchServerStats(fetchCtx, inputs.AliveServers)
|
||||
|
||||
// it might be nil but we propagate the ctx.Err just in case our context was
|
||||
// cancelled since the last time we checked.
|
||||
return inputs, ctx.Err()
|
||||
}
|
||||
|
||||
// nextState will gather many inputs about the current state of servers from the
|
||||
// delegate, raft and time provider among other sources and then compute the
|
||||
// next Autopilot state.
|
||||
func (a *Autopilot) nextState(ctx context.Context) (*State, error) {
|
||||
inputs, err := a.gatherNextStateInputs(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
state := a.nextStateWithInputs(inputs)
|
||||
if state.Leader == "" {
|
||||
return nil, fmt.Errorf("Unabled to detect the leader server")
|
||||
}
|
||||
return state, nil
|
||||
}
|
||||
|
||||
// nextStateWithInputs computes the next state given pre-gathered inputs
|
||||
func (a *Autopilot) nextStateWithInputs(inputs *nextStateInputs) *State {
|
||||
nextServers := a.nextServers(inputs)
|
||||
|
||||
newState := &State{
|
||||
startTime: inputs.StartTime,
|
||||
Healthy: true,
|
||||
Servers: nextServers,
|
||||
}
|
||||
|
||||
voterCount := 0
|
||||
healthyVoters := 0
|
||||
|
||||
// This loop will
|
||||
// 1. Determine the ID of the leader server and set it in the state
|
||||
// 2. Count the number of voters in the cluster
|
||||
// 3. Count the number of healthy voters in the cluster
|
||||
// 4. Detect unhealthy servers and mark the overall health as false
|
||||
for id, srv := range nextServers {
|
||||
if !srv.Health.Healthy {
|
||||
// any unhealthiness results in overall unhealthiness
|
||||
newState.Healthy = false
|
||||
}
|
||||
|
||||
switch srv.State {
|
||||
case RaftLeader:
|
||||
newState.Leader = id
|
||||
fallthrough
|
||||
case RaftVoter:
|
||||
newState.Voters = append(newState.Voters, id)
|
||||
voterCount++
|
||||
|
||||
if srv.Health.Healthy {
|
||||
healthyVoters++
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// If we have extra healthy voters, update FailureTolerance from its
|
||||
// zero value in the struct.
|
||||
requiredQuorum := requiredQuorum(voterCount)
|
||||
if healthyVoters > requiredQuorum {
|
||||
newState.FailureTolerance = healthyVoters - requiredQuorum
|
||||
}
|
||||
|
||||
// update any promoter specific overall state
|
||||
if newExt := a.promoter.GetStateExt(inputs.Config, newState); newExt != nil {
|
||||
newState.Ext = newExt
|
||||
}
|
||||
|
||||
// update the node types - these are really informational for users to
|
||||
// know how autopilot and the associate promoter algorithms have classed
|
||||
// each server as some promotion algorithms may want to keep certain
|
||||
// servers as non-voters for reasons. The node type then can be used
|
||||
// to indicate why that might be happening.
|
||||
for id, typ := range a.promoter.GetNodeTypes(inputs.Config, newState) {
|
||||
if srv, ok := newState.Servers[id]; ok {
|
||||
srv.Server.NodeType = typ
|
||||
}
|
||||
}
|
||||
|
||||
// Sort the voters list to keep the output stable. This is done near the end
|
||||
// as SortServers may use other parts of the state that were created in
|
||||
// this method and populated in the newState. Requiring output stability
|
||||
// helps make tests easier to manage and means that if you happen to be dumping
|
||||
// the state periodically you shouldn't see things change unless there
|
||||
// are real changes to server health or overall configuration.
|
||||
SortServers(newState.Voters, newState)
|
||||
|
||||
return newState
|
||||
}
|
||||
|
||||
// nextServers will build out the servers map for the next state to be created
|
||||
// from the given inputs. This will take into account all the various sources
|
||||
// of partial state (current state, raft config, application known servers etc.)
|
||||
// and combine them into the final server map.
|
||||
func (a *Autopilot) nextServers(inputs *nextStateInputs) map[raft.ServerID]*ServerState {
|
||||
newServers := make(map[raft.ServerID]*ServerState)
|
||||
|
||||
for _, srv := range inputs.RaftConfig.Servers {
|
||||
state := buildServerState(inputs, srv)
|
||||
|
||||
// update any promoter specific information. This isn't done within
|
||||
// buildServerState to keep that function "pure" and not require
|
||||
// mocking for tests
|
||||
if newExt := a.promoter.GetServerExt(inputs.Config, &state); newExt != nil {
|
||||
state.Server.Ext = newExt
|
||||
}
|
||||
|
||||
newServers[srv.ID] = &state
|
||||
}
|
||||
|
||||
return newServers
|
||||
}
|
||||
|
||||
// buildServerState takes all the nextStateInputs and builds out a ServerState
|
||||
// for the given Raft server. This will take into account the raft configuration
|
||||
// existing state, application known servers and recently fetched stats.
|
||||
func buildServerState(inputs *nextStateInputs, srv raft.Server) ServerState {
|
||||
// Note that the ordering of operations in this method are very important.
|
||||
// We are building up the ServerState from the least important sources
|
||||
// and overriding them with more up to date values.
|
||||
|
||||
// build the basic state from the Raft server
|
||||
state := ServerState{
|
||||
Server: Server{
|
||||
ID: srv.ID,
|
||||
Address: srv.Address,
|
||||
},
|
||||
}
|
||||
|
||||
switch srv.Suffrage {
|
||||
case raft.Voter:
|
||||
state.State = RaftVoter
|
||||
case raft.Nonvoter:
|
||||
state.State = RaftNonVoter
|
||||
case raft.Staging:
|
||||
state.State = RaftStaging
|
||||
default:
|
||||
// should be impossible unless the constants in Raft were updated
|
||||
// to have a new state.
|
||||
// TODO (mkeeler) maybe a panic would be better here. The downside is
|
||||
// that it would be hard to catch that in tests when updating the Raft
|
||||
// version.
|
||||
state.State = RaftNone
|
||||
}
|
||||
|
||||
// overwrite the raft state to mark the leader as such instead of just
|
||||
// a regular voter
|
||||
if srv.ID == inputs.LeaderID {
|
||||
state.State = RaftLeader
|
||||
}
|
||||
|
||||
var existingHealth *ServerHealth
|
||||
|
||||
// copy some state from an existing server into the new state - most of this
|
||||
// should be overridden soon but at this point we are just building the base.
|
||||
if existing, found := inputs.State.Servers[srv.ID]; found {
|
||||
state.Stats = existing.Stats
|
||||
state.Health = existing.Health
|
||||
existingHealth = &existing.Health
|
||||
|
||||
// it is is important to note that the map values we retrieved this from are
|
||||
// stored by value. Therefore we are modifying a copy of what is in the existing
|
||||
// state and not the actual state itself. We want to ensure that the Address
|
||||
// is what Raft will know about.
|
||||
existing.Server.Address = srv.Address
|
||||
|
||||
state.Server = existing.Server
|
||||
}
|
||||
|
||||
// pull in the latest information from the applications knowledge of the
|
||||
// server. Mainly we want the NodeStatus & Meta
|
||||
if known, found := inputs.KnownServers[srv.ID]; found {
|
||||
// it is important to note that we are modifying a copy of a Server as the
|
||||
// map we retrieved this from has a non-pointer type value. We definitely
|
||||
// do not want to modify the current known servers but we do want to ensure
|
||||
// that we do not overwrite the Address
|
||||
known.Address = srv.Address
|
||||
state.Server = *known
|
||||
} else {
|
||||
// TODO (mkeeler) do we need a None state. In the previous autopilot code
|
||||
// we would have set this to serf.StatusNone
|
||||
state.Server.NodeStatus = NodeLeft
|
||||
}
|
||||
|
||||
// override the Stats if any where in the fetched results
|
||||
if stats, found := inputs.FetchedStats[srv.ID]; found {
|
||||
state.Stats = *stats
|
||||
}
|
||||
|
||||
// now populate the healthy field given the stats
|
||||
state.Health.Healthy = state.isHealthy(inputs.LastTerm, inputs.LatestIndex, inputs.Config)
|
||||
// overwrite the StableSince field if this is a new server or when
|
||||
// the health status changes. No need for an else as we previously set
|
||||
// it when we overwrote the whole Health structure when finding a
|
||||
// server in the existing state
|
||||
if existingHealth == nil || existingHealth.Healthy != state.Health.Healthy {
|
||||
state.Health.StableSince = inputs.Now
|
||||
}
|
||||
|
||||
return state
|
||||
}
|
||||
|
||||
// updateState will compute the nextState, set it on the Autopilot instance and
|
||||
// then notify the delegate of the update.
|
||||
func (a *Autopilot) updateState(ctx context.Context) {
|
||||
newState, err := a.nextState(ctx)
|
||||
if err != nil {
|
||||
a.logger.Error("Error when computing next state", "error", err)
|
||||
return
|
||||
}
|
||||
|
||||
a.stateLock.Lock()
|
||||
defer a.stateLock.Unlock()
|
||||
a.state = newState
|
||||
a.delegate.NotifyState(newState)
|
||||
}
|
||||
|
||||
// SortServers will take a list of raft ServerIDs and sort it using
|
||||
// information from the State. See the ServerLessThan function for
|
||||
// details about how two servers get compared.
|
||||
func SortServers(ids []raft.ServerID, s *State) {
|
||||
sort.Slice(ids, func(i, j int) bool {
|
||||
return ServerLessThan(ids[i], ids[j], s)
|
||||
})
|
||||
}
|
||||
|
||||
// ServerLessThan will lookup both servers in the given State and return
|
||||
// true if the first id corresponds to a server that is logically less than
|
||||
// lower than, better than etc. the second server. The following criteria
|
||||
// are considered in order of most important to least important
|
||||
//
|
||||
// 1. A Leader server is always less than all others
|
||||
// 2. A voter is less than non voters
|
||||
// 3. Healthy servers are less than unhealthy servers
|
||||
// 4. Servers that have been stable longer are consider less than.
|
||||
func ServerLessThan(id1 raft.ServerID, id2 raft.ServerID, s *State) bool {
|
||||
srvI := s.Servers[id1]
|
||||
srvJ := s.Servers[id2]
|
||||
|
||||
// the leader always comes first
|
||||
if srvI.State == RaftLeader {
|
||||
return true
|
||||
} else if srvJ.State == RaftLeader {
|
||||
return false
|
||||
}
|
||||
|
||||
// voters come before non-voters & staging
|
||||
if srvI.State == RaftVoter && srvJ.State != RaftVoter {
|
||||
return true
|
||||
} else if srvI.State != RaftVoter && srvJ.State == RaftVoter {
|
||||
return false
|
||||
}
|
||||
|
||||
// at this point we know that the raft state of both nodes is roughly
|
||||
// equivalent so we want to now sort based on health
|
||||
if srvI.Health.Healthy == srvJ.Health.Healthy {
|
||||
if srvI.Health.StableSince.Before(srvJ.Health.StableSince) {
|
||||
return srvI.Health.Healthy
|
||||
} else if srvJ.Health.StableSince.Before(srvI.Health.StableSince) {
|
||||
return !srvI.Health.Healthy
|
||||
}
|
||||
|
||||
// with all else equal sort by the IDs
|
||||
return id1 < id2
|
||||
}
|
||||
|
||||
// one of the two isn't healthy. We consider the healthy one as less than
|
||||
// the other. So we return true if server I is healthy and false if it isn't
|
||||
// as we know that server J is healthy and thus should come before server I.
|
||||
return srvI.Health.Healthy
|
||||
}
|
|
@ -0,0 +1,298 @@
|
|||
package autopilot
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/hashicorp/raft"
|
||||
)
|
||||
|
||||
//go:generate mockery -all -inpkg -case snake -testonly
|
||||
|
||||
// RaftState is the status of a single server in the Raft cluster.
|
||||
type RaftState string
|
||||
|
||||
const (
|
||||
RaftNone RaftState = "none"
|
||||
RaftLeader RaftState = "leader"
|
||||
RaftVoter RaftState = "voter"
|
||||
RaftNonVoter RaftState = "non-voter"
|
||||
RaftStaging RaftState = "staging"
|
||||
)
|
||||
|
||||
func (s RaftState) IsPotentialVoter() bool {
|
||||
switch s {
|
||||
case RaftVoter, RaftStaging, RaftLeader:
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
// NodeStatus represents the health of a server as know to the autopilot consumer.
|
||||
// This should not take into account Raft health and the server being on a new enough
|
||||
// term and index.
|
||||
type NodeStatus string
|
||||
|
||||
const (
|
||||
NodeUnknown NodeStatus = "unknown"
|
||||
NodeAlive NodeStatus = "alive"
|
||||
NodeFailed NodeStatus = "failed"
|
||||
NodeLeft NodeStatus = "left"
|
||||
)
|
||||
|
||||
type NodeType string
|
||||
|
||||
const (
|
||||
NodeVoter NodeType = "voter"
|
||||
)
|
||||
|
||||
// Config represents all the tunables of autopilot
|
||||
type Config struct {
|
||||
// CleanupDeadServers controls whether to remove dead servers when a new
|
||||
// server is added to the Raft peers.
|
||||
CleanupDeadServers bool
|
||||
|
||||
// LastContactThreshold is the limit on the amount of time a server can go
|
||||
// without leader contact before being considered unhealthy.
|
||||
LastContactThreshold time.Duration
|
||||
|
||||
// MaxTrailingLogs is the amount of entries in the Raft Log that a server can
|
||||
// be behind before being considered unhealthy.
|
||||
MaxTrailingLogs uint64
|
||||
|
||||
// MinQuorum sets the minimum number of servers required in a cluster
|
||||
// before autopilot can prune dead servers.
|
||||
MinQuorum uint
|
||||
|
||||
// ServerStabilizationTime is the minimum amount of time a server must be
|
||||
// in a stable, healthy state before it can be added to the cluster. Only
|
||||
// applicable with Raft protocol version 3 or higher.
|
||||
ServerStabilizationTime time.Duration
|
||||
|
||||
Ext interface{}
|
||||
}
|
||||
|
||||
// Server represents one Raft server
|
||||
type Server struct {
|
||||
// This first set of fields are those that the autopilot delegate
|
||||
// needs to fill in
|
||||
|
||||
ID raft.ServerID
|
||||
Name string
|
||||
Address raft.ServerAddress
|
||||
NodeStatus NodeStatus
|
||||
Version string
|
||||
Meta map[string]string
|
||||
RaftVersion int
|
||||
|
||||
// The remaining fields are those that the promoter
|
||||
// will fill in
|
||||
|
||||
NodeType NodeType
|
||||
Ext interface{}
|
||||
}
|
||||
|
||||
type ServerState struct {
|
||||
Server Server
|
||||
State RaftState
|
||||
Stats ServerStats
|
||||
Health ServerHealth
|
||||
}
|
||||
|
||||
func (s *ServerState) HasVotingRights() bool {
|
||||
return s.State == RaftVoter || s.State == RaftLeader
|
||||
}
|
||||
|
||||
// isHealthy determines whether this ServerState is considered healthy
|
||||
// based on the given Autopilot config
|
||||
func (s *ServerState) isHealthy(lastTerm uint64, leaderLastIndex uint64, conf *Config) bool {
|
||||
if s.Server.NodeStatus != NodeAlive {
|
||||
return false
|
||||
}
|
||||
|
||||
if s.Stats.LastContact > conf.LastContactThreshold || s.Stats.LastContact < 0 {
|
||||
return false
|
||||
}
|
||||
|
||||
if s.Stats.LastTerm != lastTerm {
|
||||
return false
|
||||
}
|
||||
|
||||
if leaderLastIndex > conf.MaxTrailingLogs && s.Stats.LastIndex < leaderLastIndex-conf.MaxTrailingLogs {
|
||||
return false
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
type ServerHealth struct {
|
||||
// Healthy is whether or not the server is healthy according to the current
|
||||
// Autopilot config.
|
||||
Healthy bool
|
||||
|
||||
// StableSince is the last time this server's Healthy value changed.
|
||||
StableSince time.Time
|
||||
}
|
||||
|
||||
// IsStable returns true if the ServerState shows a stable, passing state
|
||||
// according to the given AutopilotConfig
|
||||
func (h *ServerHealth) IsStable(now time.Time, minStableDuration time.Duration) bool {
|
||||
if h == nil {
|
||||
return false
|
||||
}
|
||||
|
||||
if !h.Healthy {
|
||||
return false
|
||||
}
|
||||
|
||||
if now.Sub(h.StableSince) < minStableDuration {
|
||||
return false
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
// ServerStats holds miscellaneous Raft metrics for a server
|
||||
type ServerStats struct {
|
||||
// LastContact is the time since this node's last contact with the leader.
|
||||
LastContact time.Duration
|
||||
|
||||
// LastTerm is the highest leader term this server has a record of in its Raft log.
|
||||
LastTerm uint64
|
||||
|
||||
// LastIndex is the last log index this server has a record of in its Raft log.
|
||||
LastIndex uint64
|
||||
}
|
||||
|
||||
type State struct {
|
||||
startTime time.Time
|
||||
Healthy bool
|
||||
FailureTolerance int
|
||||
Servers map[raft.ServerID]*ServerState
|
||||
Leader raft.ServerID
|
||||
Voters []raft.ServerID
|
||||
Ext interface{}
|
||||
}
|
||||
|
||||
func (s *State) ServerStabilizationTime(c *Config) time.Duration {
|
||||
// Only use the configured stabilization time when autopilot has
|
||||
// been running for 110% of the configured stabilization time.
|
||||
// Before that time we haven't been running long enough to
|
||||
// be able to take these values into account. 110% is pretty
|
||||
// arbitrary but with the default config would prevent the
|
||||
// stabilization time from mattering for an extra second. This
|
||||
// allows for leeway in how quickly we get the healthy RPC responses
|
||||
// after autopilot is started.
|
||||
if time.Since(s.startTime) > (c.ServerStabilizationTime*110)/100 {
|
||||
return c.ServerStabilizationTime
|
||||
}
|
||||
|
||||
// ignore stabilization time if autopilot hasn't been running long enough
|
||||
// to be tracking any server long enough to meet that requirement
|
||||
return 0
|
||||
}
|
||||
|
||||
// Raft is the interface of all the methods on the Raft type that autopilot needs to function. Autopilot will
|
||||
// take in an interface for Raft instead of a concrete type to allow for dependency injection in tests.
|
||||
type Raft interface {
|
||||
AddNonvoter(id raft.ServerID, address raft.ServerAddress, prevIndex uint64, timeout time.Duration) raft.IndexFuture
|
||||
AddVoter(id raft.ServerID, address raft.ServerAddress, prevIndex uint64, timeout time.Duration) raft.IndexFuture
|
||||
DemoteVoter(id raft.ServerID, prevIndex uint64, timeout time.Duration) raft.IndexFuture
|
||||
LastIndex() uint64
|
||||
Leader() raft.ServerAddress
|
||||
GetConfiguration() raft.ConfigurationFuture
|
||||
RemoveServer(id raft.ServerID, prevIndex uint64, timeout time.Duration) raft.IndexFuture
|
||||
Stats() map[string]string
|
||||
LeadershipTransferToServer(id raft.ServerID, address raft.ServerAddress) raft.Future
|
||||
}
|
||||
|
||||
type ApplicationIntegration interface {
|
||||
// AutopilotConfig is used to retrieve the latest configuration from the delegate
|
||||
AutopilotConfig() *Config
|
||||
|
||||
// NotifyState will be called when the autopilot state is updated. The application may choose to emit metrics
|
||||
// or perform other actions based on this information.
|
||||
NotifyState(*State)
|
||||
|
||||
// FetchServerStats will be called to request the application fetch the ServerStats out of band. Usually this
|
||||
// will require an RPC to each server.
|
||||
FetchServerStats(context.Context, map[raft.ServerID]*Server) map[raft.ServerID]*ServerStats
|
||||
|
||||
// KnownServers fetchs the list of servers as known to the application
|
||||
KnownServers() map[raft.ServerID]*Server
|
||||
|
||||
// RemoveFailedServer notifies the application to forcefully remove the server in the failed state
|
||||
// It is expected that this returns nearly immediately so if a longer running operation needs to be
|
||||
// performed then the Delegate implementation should spawn a go routine itself.
|
||||
RemoveFailedServer(*Server)
|
||||
}
|
||||
|
||||
type RaftChanges struct {
|
||||
Promotions []raft.ServerID
|
||||
Demotions []raft.ServerID
|
||||
Leader raft.ServerID
|
||||
}
|
||||
|
||||
type FailedServers struct {
|
||||
// StaleNonVoters are the ids of those server in the raft configuration as non-voters
|
||||
// that are not present in the delegates view of what servers should be available
|
||||
StaleNonVoters []raft.ServerID
|
||||
// StaleVoters are the ids of those servers in the raft configuration as voters that
|
||||
// are not present in the delegates view of what servers should be available
|
||||
StaleVoters []raft.ServerID
|
||||
|
||||
// FailedNonVoters are the servers without voting rights in the cluster that the
|
||||
// delegate has indicated are in a failed state
|
||||
FailedNonVoters []*Server
|
||||
// FailedVoters are the servers without voting rights in the cluster that the
|
||||
// delegate has indicated are in a failed state
|
||||
FailedVoters []*Server
|
||||
}
|
||||
|
||||
// Promoter is an interface to provide promotion/demotion algorithms to the core autopilot type.
|
||||
// The BasicPromoter satisfies this interface and will promote any stable servers but other
|
||||
// algorithms could be implemented. The implementation of these methods shouldn't "block".
|
||||
// While they are synchronous autopilot expects the algorithms to not make any network
|
||||
// or other requests which way cause an indefinite amount of waiting to occur.
|
||||
//
|
||||
// Note that all parameters passed to these functions should be considered read-only and
|
||||
// their modification could result in undefined behavior of the core autopilot routines
|
||||
// including potential crashes.
|
||||
type Promoter interface {
|
||||
// GetServerExt returns some object that should be stored in the Ext field of the Server
|
||||
// This value will not be used by the code in this repo but may be used by the other
|
||||
// Promoter methods and the application utilizing autopilot. If the value returned is
|
||||
// nil the extended state will not be updated.
|
||||
GetServerExt(*Config, *ServerState) interface{}
|
||||
|
||||
// GetStateExt returns some object that should be stored in the Ext field of the State
|
||||
// This value will not be used by the code in this repo but may be used by the other
|
||||
// Promoter methods and the application utilizing autopilot. If the value returned is
|
||||
// nil the extended state will not be updated.
|
||||
GetStateExt(*Config, *State) interface{}
|
||||
|
||||
// GetNodeTypes returns a map of ServerID to NodeType for all the servers which
|
||||
// should have their NodeType field updated
|
||||
GetNodeTypes(*Config, *State) map[raft.ServerID]NodeType
|
||||
|
||||
// CalculatePromotionsAndDemotions
|
||||
CalculatePromotionsAndDemotions(*Config, *State) RaftChanges
|
||||
|
||||
// FilterFailedServerRemovals takes in the current state and structure outlining all the
|
||||
// failed/stale servers and will return those failed servers which the promoter thinks
|
||||
// should be allowed to be removed.
|
||||
FilterFailedServerRemovals(*Config, *State, *FailedServers) *FailedServers
|
||||
}
|
||||
|
||||
// timeProvider is an interface for getting a local time. This is mainly useful for testing
|
||||
// to inject certain times so that output validation is easier.
|
||||
type timeProvider interface {
|
||||
Now() time.Time
|
||||
}
|
||||
|
||||
type runtimeTimeProvider struct{}
|
||||
|
||||
func (_ *runtimeTimeProvider) Now() time.Time {
|
||||
return time.Now()
|
||||
}
|
|
@ -10,6 +10,7 @@ matrix:
|
|||
include:
|
||||
- go: 1.12.x
|
||||
- go: 1.13.x
|
||||
- go: 1.14.x
|
||||
env: LINT=1
|
||||
|
||||
install:
|
||||
|
|
|
@ -4,6 +4,12 @@ All notable changes to this project will be documented in this file.
|
|||
The format is based on [Keep a Changelog](http://keepachangelog.com/en/1.0.0/)
|
||||
and this project adheres to [Semantic Versioning](http://semver.org/spec/v2.0.0.html).
|
||||
|
||||
## [1.1.0]
|
||||
### Added
|
||||
- [#49]: Add option to ignore current goroutines, which checks for any additional leaks and allows for incremental adoption of goleak in larger projects.
|
||||
|
||||
Thanks to @denis-tingajkin for their contributions to this release.
|
||||
|
||||
## [1.0.0]
|
||||
### Changed
|
||||
- Migrate to Go modules.
|
||||
|
@ -15,3 +21,4 @@ and this project adheres to [Semantic Versioning](http://semver.org/spec/v2.0.0.
|
|||
- Initial release.
|
||||
|
||||
[1.0.0]: https://github.com/uber-go/goleak/compare/v0.10.0...v1.0.0
|
||||
[#49]: https://github.com/uber-go/goleak/pull/49
|
||||
|
|
|
@ -2,10 +2,6 @@
|
|||
|
||||
Goroutine leak detector to help avoid Goroutine leaks.
|
||||
|
||||
## Development Status: Alpha
|
||||
|
||||
goleak is still in development, and APIs are still in flux.
|
||||
|
||||
## Installation
|
||||
|
||||
You can use `go get` to get the latest version:
|
||||
|
@ -50,7 +46,7 @@ $ go test -c -o tests
|
|||
|
||||
# Run each test individually, printing "." for successful tests, or the test name
|
||||
# for failing tests.
|
||||
$ for test in $(go test -list . | grep "^Test"); do ./tests -test.run "^$test\$" &>/dev/null && echo -n "." || echo "\n$test failed"; done
|
||||
$ for test in $(go test -list . | grep -E "^(Test|Example)"); do ./tests -test.run "^$test\$" &>/dev/null && echo -n "." || echo "\n$test failed"; done
|
||||
```
|
||||
|
||||
This will only print names of failing tests which can be investigated individually. E.g.,
|
||||
|
@ -61,6 +57,11 @@ TestLeakyTest failed
|
|||
.......
|
||||
```
|
||||
|
||||
## Stability
|
||||
|
||||
goleak is v1 and follows [SemVer](http://semver.org/) strictly.
|
||||
|
||||
No breaking changes will be made to exported APIs before 2.0.
|
||||
|
||||
[doc-img]: https://godoc.org/go.uber.org/goleak?status.svg
|
||||
[doc]: https://godoc.org/go.uber.org/goleak
|
||||
|
|
|
@ -57,6 +57,18 @@ func IgnoreTopFunction(f string) Option {
|
|||
})
|
||||
}
|
||||
|
||||
// IgnoreCurrent records all current goroutines when the option is created, and ignores
|
||||
// them in any future Find/Verify calls.
|
||||
func IgnoreCurrent() Option {
|
||||
excludeIDSet := map[int]bool{}
|
||||
for _, s := range stack.All() {
|
||||
excludeIDSet[s.ID()] = true
|
||||
}
|
||||
return addFilter(func(s stack.Stack) bool {
|
||||
return excludeIDSet[s.ID()]
|
||||
})
|
||||
}
|
||||
|
||||
func maxSleep(d time.Duration) Option {
|
||||
return optionFunc(func(opts *opts) {
|
||||
opts.maxSleep = d
|
||||
|
|
|
@ -227,7 +227,7 @@ github.com/hashicorp/go-discover/provider/softlayer
|
|||
github.com/hashicorp/go-discover/provider/tencentcloud
|
||||
github.com/hashicorp/go-discover/provider/triton
|
||||
github.com/hashicorp/go-discover/provider/vsphere
|
||||
# github.com/hashicorp/go-hclog v0.12.0
|
||||
# github.com/hashicorp/go-hclog v0.14.1
|
||||
github.com/hashicorp/go-hclog
|
||||
# github.com/hashicorp/go-immutable-radix v1.3.0
|
||||
github.com/hashicorp/go-immutable-radix
|
||||
|
@ -280,6 +280,8 @@ github.com/hashicorp/memberlist
|
|||
github.com/hashicorp/net-rpc-msgpackrpc
|
||||
# github.com/hashicorp/raft v1.2.0
|
||||
github.com/hashicorp/raft
|
||||
# github.com/hashicorp/raft-autopilot v0.1.1
|
||||
github.com/hashicorp/raft-autopilot
|
||||
# github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea
|
||||
github.com/hashicorp/raft-boltdb
|
||||
# github.com/hashicorp/serf v0.9.5
|
||||
|
@ -456,7 +458,7 @@ go.opencensus.io/trace
|
|||
go.opencensus.io/trace/internal
|
||||
go.opencensus.io/trace/propagation
|
||||
go.opencensus.io/trace/tracestate
|
||||
# go.uber.org/goleak v1.0.0
|
||||
# go.uber.org/goleak v1.1.10
|
||||
go.uber.org/goleak
|
||||
go.uber.org/goleak/internal/stack
|
||||
# golang.org/x/crypto v0.0.0-20200930160638-afb6bcd081ae
|
||||
|
|
|
@ -17,6 +17,15 @@ upgrade flow.
|
|||
|
||||
## Consul 1.9.0
|
||||
|
||||
### Changes to Raft Protocol Support
|
||||
|
||||
Consul 1.8 supported Raft protocols 2 and 3. Consul 1.9.0 now only supports
|
||||
Raft protocol 3 so before upgrading to Consul 1.9.0 users may have to first
|
||||
upgrade to a previous release supporting both protocol versions and upgrade
|
||||
the protocol in use to version 3.
|
||||
|
||||
### Changes to Configuration Defaults
|
||||
|
||||
The [`enable_central_service_config`](/docs/agent/options#enable_central_service_config)
|
||||
configuration now defaults to `true`.
|
||||
|
||||
|
|
Loading…
Reference in New Issue