2015-06-01 15:49:10 +00:00
|
|
|
package nomad
|
|
|
|
|
2015-06-04 11:38:41 +00:00
|
|
|
import (
|
2017-08-30 17:06:56 +00:00
|
|
|
"bytes"
|
2016-08-22 20:57:27 +00:00
|
|
|
"context"
|
2015-06-04 11:38:41 +00:00
|
|
|
"fmt"
|
2017-04-12 21:47:59 +00:00
|
|
|
"math/rand"
|
2017-02-02 23:49:06 +00:00
|
|
|
"net"
|
2019-11-14 13:18:29 +00:00
|
|
|
"strings"
|
2017-10-17 05:01:49 +00:00
|
|
|
"sync"
|
2015-06-04 11:38:41 +00:00
|
|
|
"time"
|
|
|
|
|
2017-08-13 23:16:59 +00:00
|
|
|
"golang.org/x/time/rate"
|
|
|
|
|
2019-01-15 19:46:12 +00:00
|
|
|
metrics "github.com/armon/go-metrics"
|
2018-09-15 23:23:13 +00:00
|
|
|
log "github.com/hashicorp/go-hclog"
|
2017-02-08 04:31:23 +00:00
|
|
|
memdb "github.com/hashicorp/go-memdb"
|
2019-01-15 19:46:12 +00:00
|
|
|
version "github.com/hashicorp/go-version"
|
2017-09-29 16:58:48 +00:00
|
|
|
"github.com/hashicorp/nomad/helper/uuid"
|
2017-08-13 23:16:59 +00:00
|
|
|
"github.com/hashicorp/nomad/nomad/state"
|
2015-08-05 23:53:54 +00:00
|
|
|
"github.com/hashicorp/nomad/nomad/structs"
|
2015-06-04 11:38:41 +00:00
|
|
|
"github.com/hashicorp/raft"
|
|
|
|
"github.com/hashicorp/serf/serf"
|
2019-11-14 13:18:29 +00:00
|
|
|
"github.com/pkg/errors"
|
2015-06-04 11:38:41 +00:00
|
|
|
)
|
|
|
|
|
2016-05-25 17:28:25 +00:00
|
|
|
const (
|
|
|
|
// failedEvalUnblockInterval is the interval at which failed evaluations are
|
|
|
|
// unblocked to re-enter the scheduler. A failed evaluation occurs under
|
|
|
|
// high contention when the schedulers plan does not make progress.
|
|
|
|
failedEvalUnblockInterval = 1 * time.Minute
|
2017-08-13 23:16:59 +00:00
|
|
|
|
|
|
|
// replicationRateLimit is used to rate limit how often data is replicated
|
|
|
|
// between the authoritative region and the local region
|
|
|
|
replicationRateLimit rate.Limit = 10.0
|
2017-10-17 05:01:49 +00:00
|
|
|
|
|
|
|
// barrierWriteTimeout is used to give Raft a chance to process a
|
|
|
|
// possible loss of leadership event if we are unable to get a barrier
|
|
|
|
// while leader.
|
|
|
|
barrierWriteTimeout = 2 * time.Minute
|
2016-05-25 17:28:25 +00:00
|
|
|
)
|
|
|
|
|
2017-12-18 21:16:23 +00:00
|
|
|
var minAutopilotVersion = version.Must(version.NewVersion("0.8.0"))
|
|
|
|
|
2019-01-29 19:48:45 +00:00
|
|
|
var minSchedulerConfigVersion = version.Must(version.NewVersion("0.9.0"))
|
2019-01-29 18:47:42 +00:00
|
|
|
|
2019-11-14 13:18:29 +00:00
|
|
|
var minClusterIDVersion = version.Must(version.NewVersion("0.10.3"))
|
|
|
|
|
|
|
|
// Default configuration for scheduler with preemption enabled for system jobs
|
|
|
|
var defaultSchedulerConfig = &structs.SchedulerConfiguration{
|
|
|
|
PreemptionConfig: structs.PreemptionConfig{
|
|
|
|
SystemSchedulerEnabled: true,
|
|
|
|
BatchSchedulerEnabled: false,
|
|
|
|
ServiceSchedulerEnabled: false,
|
|
|
|
},
|
|
|
|
}
|
|
|
|
|
2015-06-01 15:49:10 +00:00
|
|
|
// monitorLeadership is used to monitor if we acquire or lose our role
|
|
|
|
// as the leader in the Raft cluster. There is some work the leader is
|
|
|
|
// expected to do, so we must react to changes
|
|
|
|
func (s *Server) monitorLeadership() {
|
2017-10-17 05:01:49 +00:00
|
|
|
var weAreLeaderCh chan struct{}
|
|
|
|
var leaderLoop sync.WaitGroup
|
2020-01-22 13:21:33 +00:00
|
|
|
|
|
|
|
leaderStep := func(isLeader bool) {
|
2020-01-28 14:06:52 +00:00
|
|
|
if isLeader {
|
2020-01-22 13:21:33 +00:00
|
|
|
if weAreLeaderCh != nil {
|
|
|
|
s.logger.Error("attempted to start the leader loop while running")
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
weAreLeaderCh = make(chan struct{})
|
|
|
|
leaderLoop.Add(1)
|
|
|
|
go func(ch chan struct{}) {
|
|
|
|
defer leaderLoop.Done()
|
|
|
|
s.leaderLoop(ch)
|
|
|
|
}(weAreLeaderCh)
|
|
|
|
s.logger.Info("cluster leadership acquired")
|
2020-01-28 14:06:52 +00:00
|
|
|
return
|
|
|
|
}
|
2020-01-22 13:21:33 +00:00
|
|
|
|
2020-01-28 14:06:52 +00:00
|
|
|
if weAreLeaderCh == nil {
|
|
|
|
s.logger.Error("attempted to stop the leader loop while not running")
|
|
|
|
return
|
2020-01-22 13:21:33 +00:00
|
|
|
}
|
2020-01-28 14:06:52 +00:00
|
|
|
|
|
|
|
s.logger.Debug("shutting down leader loop")
|
|
|
|
close(weAreLeaderCh)
|
|
|
|
leaderLoop.Wait()
|
|
|
|
weAreLeaderCh = nil
|
|
|
|
s.logger.Info("cluster leadership lost")
|
2020-01-22 13:21:33 +00:00
|
|
|
}
|
|
|
|
|
Handle Nomad leadership flapping
Fixes a deadlock in leadership handling if leadership flapped.
Raft propagates leadership transition to Nomad through a NotifyCh channel.
Raft blocks when writing to this channel, so channel must be buffered or
aggressively consumed[1]. Otherwise, Raft blocks indefinitely in `raft.runLeader`
until the channel is consumed[1] and does not move on to executing follower
related logic (in `raft.runFollower`).
While Raft `runLeader` defer function blocks, raft cannot process any other
raft operations. For example, `run{Leader|Follower}` methods consume
`raft.applyCh`, and while runLeader defer is blocked, all raft log applications
or config lookup will block indefinitely.
Sadly, `leaderLoop` and `establishLeader` makes few Raft calls!
`establishLeader` attempts to auto-create autopilot/scheduler config [3]; and
`leaderLoop` attempts to check raft configuration [4]. All of these calls occur
without a timeout.
Thus, if leadership flapped quickly while `leaderLoop/establishLeadership` is
invoked and hit any of these Raft calls, Raft handler _deadlock_ forever.
Depending on how many times it flapped and where exactly we get stuck, I suspect
it's possible to get in the following case:
* Agent metrics/stats http and RPC calls hang as they check raft.Configurations
* raft.State remains in Leader state, and server attempts to handle RPC calls
(e.g. node/alloc updates) and these hang as well
As we create goroutines per RPC call, the number of goroutines grow over time
and may trigger a out of memory errors in addition to missed updates.
[1] https://github.com/hashicorp/raft/blob/d90d6d6bdacf1b35d66940b07be515b074d89e88/config.go#L190-L193
[2] https://github.com/hashicorp/raft/blob/d90d6d6bdacf1b35d66940b07be515b074d89e88/raft.go#L425-L436
[3] https://github.com/hashicorp/nomad/blob/2a89e477465adbe6a88987f0dcb9fe80145d7b2f/nomad/leader.go#L198-L202
[4] https://github.com/hashicorp/nomad/blob/2a89e477465adbe6a88987f0dcb9fe80145d7b2f/nomad/leader.go#L877
2020-01-22 15:55:44 +00:00
|
|
|
wasLeader := false
|
2015-06-01 15:49:10 +00:00
|
|
|
for {
|
|
|
|
select {
|
2015-09-07 17:46:41 +00:00
|
|
|
case isLeader := <-s.leaderCh:
|
Handle Nomad leadership flapping
Fixes a deadlock in leadership handling if leadership flapped.
Raft propagates leadership transition to Nomad through a NotifyCh channel.
Raft blocks when writing to this channel, so channel must be buffered or
aggressively consumed[1]. Otherwise, Raft blocks indefinitely in `raft.runLeader`
until the channel is consumed[1] and does not move on to executing follower
related logic (in `raft.runFollower`).
While Raft `runLeader` defer function blocks, raft cannot process any other
raft operations. For example, `run{Leader|Follower}` methods consume
`raft.applyCh`, and while runLeader defer is blocked, all raft log applications
or config lookup will block indefinitely.
Sadly, `leaderLoop` and `establishLeader` makes few Raft calls!
`establishLeader` attempts to auto-create autopilot/scheduler config [3]; and
`leaderLoop` attempts to check raft configuration [4]. All of these calls occur
without a timeout.
Thus, if leadership flapped quickly while `leaderLoop/establishLeadership` is
invoked and hit any of these Raft calls, Raft handler _deadlock_ forever.
Depending on how many times it flapped and where exactly we get stuck, I suspect
it's possible to get in the following case:
* Agent metrics/stats http and RPC calls hang as they check raft.Configurations
* raft.State remains in Leader state, and server attempts to handle RPC calls
(e.g. node/alloc updates) and these hang as well
As we create goroutines per RPC call, the number of goroutines grow over time
and may trigger a out of memory errors in addition to missed updates.
[1] https://github.com/hashicorp/raft/blob/d90d6d6bdacf1b35d66940b07be515b074d89e88/config.go#L190-L193
[2] https://github.com/hashicorp/raft/blob/d90d6d6bdacf1b35d66940b07be515b074d89e88/raft.go#L425-L436
[3] https://github.com/hashicorp/nomad/blob/2a89e477465adbe6a88987f0dcb9fe80145d7b2f/nomad/leader.go#L198-L202
[4] https://github.com/hashicorp/nomad/blob/2a89e477465adbe6a88987f0dcb9fe80145d7b2f/nomad/leader.go#L877
2020-01-22 15:55:44 +00:00
|
|
|
if wasLeader != isLeader {
|
|
|
|
wasLeader = isLeader
|
|
|
|
// normal case where we went through a transition
|
|
|
|
leaderStep(isLeader)
|
|
|
|
} else if wasLeader && isLeader {
|
|
|
|
// Server lost but then gained leadership immediately.
|
|
|
|
// During this time, this server may have received
|
|
|
|
// Raft transitions that haven't been applied to the FSM
|
|
|
|
// yet.
|
|
|
|
// Ensure that that FSM caught up and eval queues are refreshed
|
2020-01-28 14:49:36 +00:00
|
|
|
s.logger.Warn("cluster leadership lost and gained leadership immediately. Could indicate network issues, memory paging, or high CPU load.")
|
Handle Nomad leadership flapping
Fixes a deadlock in leadership handling if leadership flapped.
Raft propagates leadership transition to Nomad through a NotifyCh channel.
Raft blocks when writing to this channel, so channel must be buffered or
aggressively consumed[1]. Otherwise, Raft blocks indefinitely in `raft.runLeader`
until the channel is consumed[1] and does not move on to executing follower
related logic (in `raft.runFollower`).
While Raft `runLeader` defer function blocks, raft cannot process any other
raft operations. For example, `run{Leader|Follower}` methods consume
`raft.applyCh`, and while runLeader defer is blocked, all raft log applications
or config lookup will block indefinitely.
Sadly, `leaderLoop` and `establishLeader` makes few Raft calls!
`establishLeader` attempts to auto-create autopilot/scheduler config [3]; and
`leaderLoop` attempts to check raft configuration [4]. All of these calls occur
without a timeout.
Thus, if leadership flapped quickly while `leaderLoop/establishLeadership` is
invoked and hit any of these Raft calls, Raft handler _deadlock_ forever.
Depending on how many times it flapped and where exactly we get stuck, I suspect
it's possible to get in the following case:
* Agent metrics/stats http and RPC calls hang as they check raft.Configurations
* raft.State remains in Leader state, and server attempts to handle RPC calls
(e.g. node/alloc updates) and these hang as well
As we create goroutines per RPC call, the number of goroutines grow over time
and may trigger a out of memory errors in addition to missed updates.
[1] https://github.com/hashicorp/raft/blob/d90d6d6bdacf1b35d66940b07be515b074d89e88/config.go#L190-L193
[2] https://github.com/hashicorp/raft/blob/d90d6d6bdacf1b35d66940b07be515b074d89e88/raft.go#L425-L436
[3] https://github.com/hashicorp/nomad/blob/2a89e477465adbe6a88987f0dcb9fe80145d7b2f/nomad/leader.go#L198-L202
[4] https://github.com/hashicorp/nomad/blob/2a89e477465adbe6a88987f0dcb9fe80145d7b2f/nomad/leader.go#L877
2020-01-22 15:55:44 +00:00
|
|
|
|
|
|
|
leaderStep(false)
|
|
|
|
leaderStep(true)
|
|
|
|
} else {
|
|
|
|
// Server gained but lost leadership immediately
|
|
|
|
// before it reacted; nothing to do, move on
|
2020-01-28 14:49:36 +00:00
|
|
|
s.logger.Warn("cluster leadership gained and lost leadership immediately. Could indicate network issues, memory paging, or high CPU load.")
|
Handle Nomad leadership flapping
Fixes a deadlock in leadership handling if leadership flapped.
Raft propagates leadership transition to Nomad through a NotifyCh channel.
Raft blocks when writing to this channel, so channel must be buffered or
aggressively consumed[1]. Otherwise, Raft blocks indefinitely in `raft.runLeader`
until the channel is consumed[1] and does not move on to executing follower
related logic (in `raft.runFollower`).
While Raft `runLeader` defer function blocks, raft cannot process any other
raft operations. For example, `run{Leader|Follower}` methods consume
`raft.applyCh`, and while runLeader defer is blocked, all raft log applications
or config lookup will block indefinitely.
Sadly, `leaderLoop` and `establishLeader` makes few Raft calls!
`establishLeader` attempts to auto-create autopilot/scheduler config [3]; and
`leaderLoop` attempts to check raft configuration [4]. All of these calls occur
without a timeout.
Thus, if leadership flapped quickly while `leaderLoop/establishLeadership` is
invoked and hit any of these Raft calls, Raft handler _deadlock_ forever.
Depending on how many times it flapped and where exactly we get stuck, I suspect
it's possible to get in the following case:
* Agent metrics/stats http and RPC calls hang as they check raft.Configurations
* raft.State remains in Leader state, and server attempts to handle RPC calls
(e.g. node/alloc updates) and these hang as well
As we create goroutines per RPC call, the number of goroutines grow over time
and may trigger a out of memory errors in addition to missed updates.
[1] https://github.com/hashicorp/raft/blob/d90d6d6bdacf1b35d66940b07be515b074d89e88/config.go#L190-L193
[2] https://github.com/hashicorp/raft/blob/d90d6d6bdacf1b35d66940b07be515b074d89e88/raft.go#L425-L436
[3] https://github.com/hashicorp/nomad/blob/2a89e477465adbe6a88987f0dcb9fe80145d7b2f/nomad/leader.go#L198-L202
[4] https://github.com/hashicorp/nomad/blob/2a89e477465adbe6a88987f0dcb9fe80145d7b2f/nomad/leader.go#L877
2020-01-22 15:55:44 +00:00
|
|
|
}
|
2015-06-01 15:49:10 +00:00
|
|
|
case <-s.shutdownCh:
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// leaderLoop runs as long as we are the leader to run various
|
2018-03-11 18:26:20 +00:00
|
|
|
// maintenance activities
|
2015-06-01 15:49:10 +00:00
|
|
|
func (s *Server) leaderLoop(stopCh chan struct{}) {
|
2015-06-05 21:54:45 +00:00
|
|
|
var reconcileCh chan serf.Member
|
2015-07-24 04:58:38 +00:00
|
|
|
establishedLeader := false
|
|
|
|
|
2015-06-05 21:54:45 +00:00
|
|
|
RECONCILE:
|
|
|
|
// Setup a reconciliation timer
|
|
|
|
reconcileCh = nil
|
|
|
|
interval := time.After(s.config.ReconcileInterval)
|
|
|
|
|
|
|
|
// Apply a raft barrier to ensure our FSM is caught up
|
|
|
|
start := time.Now()
|
2017-10-17 05:01:49 +00:00
|
|
|
barrier := s.raft.Barrier(barrierWriteTimeout)
|
2015-06-05 21:54:45 +00:00
|
|
|
if err := barrier.Error(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to wait for barrier", "error", err)
|
2015-06-05 21:54:45 +00:00
|
|
|
goto WAIT
|
|
|
|
}
|
|
|
|
metrics.MeasureSince([]string{"nomad", "leader", "barrier"}, start)
|
|
|
|
|
2015-07-24 04:58:38 +00:00
|
|
|
// Check if we need to handle initial leadership actions
|
|
|
|
if !establishedLeader {
|
2015-08-15 22:15:00 +00:00
|
|
|
if err := s.establishLeadership(stopCh); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to establish leadership", "error", err)
|
2018-02-20 20:52:00 +00:00
|
|
|
|
|
|
|
// Immediately revoke leadership since we didn't successfully
|
|
|
|
// establish leadership.
|
|
|
|
if err := s.revokeLeadership(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to revoke leadership", "error", err)
|
2018-02-20 20:52:00 +00:00
|
|
|
}
|
|
|
|
|
2015-07-24 04:58:38 +00:00
|
|
|
goto WAIT
|
|
|
|
}
|
2018-02-20 20:52:00 +00:00
|
|
|
|
2015-07-24 04:58:38 +00:00
|
|
|
establishedLeader = true
|
2017-10-17 05:01:49 +00:00
|
|
|
defer func() {
|
|
|
|
if err := s.revokeLeadership(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to revoke leadership", "error", err)
|
2017-10-17 05:01:49 +00:00
|
|
|
}
|
|
|
|
}()
|
2015-07-24 04:58:38 +00:00
|
|
|
}
|
|
|
|
|
2015-06-05 21:54:45 +00:00
|
|
|
// Reconcile any missing data
|
|
|
|
if err := s.reconcile(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to reconcile", "error", err)
|
2015-06-05 21:54:45 +00:00
|
|
|
goto WAIT
|
|
|
|
}
|
|
|
|
|
|
|
|
// Initial reconcile worked, now we can process the channel
|
|
|
|
// updates
|
|
|
|
reconcileCh = s.reconcileCh
|
|
|
|
|
2017-10-17 05:01:49 +00:00
|
|
|
// Poll the stop channel to give it priority so we don't waste time
|
|
|
|
// trying to perform the other operations if we have been asked to shut
|
|
|
|
// down.
|
|
|
|
select {
|
|
|
|
case <-stopCh:
|
|
|
|
return
|
|
|
|
default:
|
|
|
|
}
|
|
|
|
|
2015-06-05 21:54:45 +00:00
|
|
|
WAIT:
|
2015-06-01 15:49:10 +00:00
|
|
|
// Wait until leadership is lost
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-stopCh:
|
|
|
|
return
|
|
|
|
case <-s.shutdownCh:
|
|
|
|
return
|
2015-06-05 21:54:45 +00:00
|
|
|
case <-interval:
|
|
|
|
goto RECONCILE
|
|
|
|
case member := <-reconcileCh:
|
2015-06-04 11:38:41 +00:00
|
|
|
s.reconcileMember(member)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-07-24 04:58:38 +00:00
|
|
|
// establishLeadership is invoked once we become leader and are able
|
|
|
|
// to invoke an initial barrier. The barrier is used to ensure any
|
2016-05-15 16:41:34 +00:00
|
|
|
// previously inflight transactions have been committed and that our
|
2015-07-24 04:58:38 +00:00
|
|
|
// state is up-to-date.
|
2015-08-15 22:15:00 +00:00
|
|
|
func (s *Server) establishLeadership(stopCh chan struct{}) error {
|
2018-02-20 18:23:11 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "leader", "establish_leadership"}, time.Now())
|
|
|
|
|
2017-10-23 19:50:37 +00:00
|
|
|
// Generate a leader ACL token. This will allow the leader to issue work
|
|
|
|
// that requires a valid ACL token.
|
|
|
|
s.setLeaderAcl(uuid.Generate())
|
|
|
|
|
2016-02-17 21:50:06 +00:00
|
|
|
// Disable workers to free half the cores for use in the plan queue and
|
|
|
|
// evaluation broker
|
|
|
|
if numWorkers := len(s.workers); numWorkers > 1 {
|
2016-02-20 21:38:46 +00:00
|
|
|
// Disabling 3/4 of the workers frees CPU for raft and the
|
|
|
|
// plan applier which uses 1/2 the cores.
|
|
|
|
for i := 0; i < (3 * numWorkers / 4); i++ {
|
2016-02-17 21:50:06 +00:00
|
|
|
s.workers[i].SetPause(true)
|
|
|
|
}
|
2015-08-23 20:59:13 +00:00
|
|
|
}
|
|
|
|
|
2017-12-18 21:16:23 +00:00
|
|
|
// Initialize and start the autopilot routine
|
|
|
|
s.getOrCreateAutopilotConfig()
|
|
|
|
s.autopilot.Start()
|
|
|
|
|
2018-09-28 04:27:38 +00:00
|
|
|
// Initialize scheduler configuration
|
|
|
|
s.getOrCreateSchedulerConfig()
|
|
|
|
|
2019-11-14 13:18:29 +00:00
|
|
|
// Initialize the ClusterID
|
|
|
|
_, _ = s.ClusterID()
|
|
|
|
// todo: use cluster ID for stuff, later!
|
|
|
|
|
2015-07-27 22:11:42 +00:00
|
|
|
// Enable the plan queue, since we are now the leader
|
|
|
|
s.planQueue.SetEnabled(true)
|
|
|
|
|
2015-07-27 22:31:09 +00:00
|
|
|
// Start the plan evaluator
|
|
|
|
go s.planApply()
|
2015-07-27 22:11:42 +00:00
|
|
|
|
2015-07-24 04:58:38 +00:00
|
|
|
// Enable the eval broker, since we are now the leader
|
|
|
|
s.evalBroker.SetEnabled(true)
|
|
|
|
|
2016-01-29 23:31:32 +00:00
|
|
|
// Enable the blocked eval tracker, since we are now the leader
|
|
|
|
s.blockedEvals.SetEnabled(true)
|
2017-10-13 21:36:02 +00:00
|
|
|
s.blockedEvals.SetTimetable(s.fsm.TimeTable())
|
2016-01-29 23:31:32 +00:00
|
|
|
|
2017-06-28 22:35:52 +00:00
|
|
|
// Enable the deployment watcher, since we are now the leader
|
2018-02-27 00:28:10 +00:00
|
|
|
s.deploymentWatcher.SetEnabled(true, s.State())
|
|
|
|
|
|
|
|
// Enable the NodeDrainer
|
|
|
|
s.nodeDrainer.SetEnabled(true, s.State())
|
2017-06-28 22:35:52 +00:00
|
|
|
|
2015-08-05 23:53:54 +00:00
|
|
|
// Restore the eval broker state
|
2016-01-29 23:31:32 +00:00
|
|
|
if err := s.restoreEvals(); err != nil {
|
2015-08-05 23:53:54 +00:00
|
|
|
return err
|
|
|
|
}
|
2015-08-15 22:15:00 +00:00
|
|
|
|
2016-08-22 20:57:27 +00:00
|
|
|
// Activate the vault client
|
|
|
|
s.vault.SetActive(true)
|
2019-12-06 20:46:46 +00:00
|
|
|
// Cleanup orphaned Vault token accessors
|
|
|
|
if err := s.revokeVaultAccessorsOnRestore(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Cleanup orphaned Service Identity token accessors
|
|
|
|
if err := s.revokeSITokenAccessorsOnRestore(); err != nil {
|
2016-08-22 20:57:27 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2015-12-04 23:10:08 +00:00
|
|
|
// Enable the periodic dispatcher, since we are now the leader.
|
2015-12-18 20:26:28 +00:00
|
|
|
s.periodicDispatcher.SetEnabled(true)
|
|
|
|
|
|
|
|
// Restore the periodic dispatcher state
|
|
|
|
if err := s.restorePeriodicDispatcher(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2015-08-15 22:15:00 +00:00
|
|
|
// Scheduler periodic jobs
|
|
|
|
go s.schedulePeriodic(stopCh)
|
2015-08-16 18:10:18 +00:00
|
|
|
|
|
|
|
// Reap any failed evaluations
|
|
|
|
go s.reapFailedEvaluations(stopCh)
|
2015-08-23 00:17:13 +00:00
|
|
|
|
2016-01-31 00:16:13 +00:00
|
|
|
// Reap any duplicate blocked evaluations
|
|
|
|
go s.reapDupBlockedEvaluations(stopCh)
|
|
|
|
|
2016-05-23 23:27:26 +00:00
|
|
|
// Periodically unblock failed allocations
|
|
|
|
go s.periodicUnblockFailedEvals(stopCh)
|
|
|
|
|
2017-10-30 19:19:11 +00:00
|
|
|
// Periodically publish job summary metrics
|
|
|
|
go s.publishJobSummaryMetrics(stopCh)
|
|
|
|
|
2019-07-24 13:17:33 +00:00
|
|
|
// Periodically publish job status metrics
|
|
|
|
go s.publishJobStatusMetrics(stopCh)
|
|
|
|
|
2015-08-23 00:17:13 +00:00
|
|
|
// Setup the heartbeat timers. This is done both when starting up or when
|
|
|
|
// a leader fail over happens. Since the timers are maintained by the leader
|
|
|
|
// node, effectively this means all the timers are renewed at the time of failover.
|
|
|
|
// The TTL contract is that the session will not be expired before the TTL,
|
|
|
|
// so expiring it later is allowable.
|
|
|
|
//
|
|
|
|
// This MUST be done after the initial barrier to ensure the latest Nodes
|
|
|
|
// are available to be initialized. Otherwise initialization may use stale
|
|
|
|
// data.
|
|
|
|
if err := s.initializeHeartbeatTimers(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("heartbeat timer setup failed", "error", err)
|
2015-08-23 00:17:13 +00:00
|
|
|
return err
|
|
|
|
}
|
2016-08-05 21:40:35 +00:00
|
|
|
|
2017-08-13 23:16:59 +00:00
|
|
|
// Start replication of ACLs and Policies if they are enabled,
|
|
|
|
// and we are not the authoritative region.
|
|
|
|
if s.config.ACLEnabled && s.config.Region != s.config.AuthoritativeRegion {
|
|
|
|
go s.replicateACLPolicies(stopCh)
|
|
|
|
go s.replicateACLTokens(stopCh)
|
|
|
|
}
|
2017-09-07 23:56:15 +00:00
|
|
|
|
|
|
|
// Setup any enterprise systems required.
|
2017-09-13 18:38:29 +00:00
|
|
|
if err := s.establishEnterpriseLeadership(stopCh); err != nil {
|
2017-09-07 23:56:15 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2019-07-02 07:58:02 +00:00
|
|
|
s.setConsistentReadReady()
|
|
|
|
|
2015-08-05 23:53:54 +00:00
|
|
|
return nil
|
|
|
|
}
|
2015-07-27 22:11:42 +00:00
|
|
|
|
2016-01-29 23:31:32 +00:00
|
|
|
// restoreEvals is used to restore pending evaluations into the eval broker and
|
|
|
|
// blocked evaluations into the blocked eval tracker. The broker and blocked
|
|
|
|
// eval tracker is maintained only by the leader, so it must be restored anytime
|
|
|
|
// a leadership transition takes place.
|
|
|
|
func (s *Server) restoreEvals() error {
|
2015-08-05 23:53:54 +00:00
|
|
|
// Get an iterator over every evaluation
|
2017-02-08 04:31:23 +00:00
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
iter, err := s.fsm.State().Evals(ws)
|
2015-08-05 23:53:54 +00:00
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("failed to get evaluations: %v", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
for {
|
|
|
|
raw := iter.Next()
|
|
|
|
if raw == nil {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
eval := raw.(*structs.Evaluation)
|
|
|
|
|
2016-01-29 23:31:32 +00:00
|
|
|
if eval.ShouldEnqueue() {
|
2016-05-18 18:35:15 +00:00
|
|
|
s.evalBroker.Enqueue(eval)
|
2016-01-29 23:31:32 +00:00
|
|
|
} else if eval.ShouldBlock() {
|
|
|
|
s.blockedEvals.Block(eval)
|
2015-08-05 23:53:54 +00:00
|
|
|
}
|
|
|
|
}
|
2015-07-24 04:58:38 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-12-06 20:46:46 +00:00
|
|
|
// revokeVaultAccessorsOnRestore is used to restore Vault accessors that should be
|
2016-08-22 20:57:27 +00:00
|
|
|
// revoked.
|
2019-12-06 20:46:46 +00:00
|
|
|
func (s *Server) revokeVaultAccessorsOnRestore() error {
|
2016-08-22 20:57:27 +00:00
|
|
|
// An accessor should be revoked if its allocation or node is terminal
|
2017-02-08 04:31:23 +00:00
|
|
|
ws := memdb.NewWatchSet()
|
2016-08-22 20:57:27 +00:00
|
|
|
state := s.fsm.State()
|
2017-02-08 04:31:23 +00:00
|
|
|
iter, err := state.VaultAccessors(ws)
|
2016-08-22 20:57:27 +00:00
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("failed to get vault accessors: %v", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
var revoke []*structs.VaultAccessor
|
|
|
|
for {
|
|
|
|
raw := iter.Next()
|
|
|
|
if raw == nil {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
va := raw.(*structs.VaultAccessor)
|
|
|
|
|
|
|
|
// Check the allocation
|
2017-02-08 04:31:23 +00:00
|
|
|
alloc, err := state.AllocByID(ws, va.AllocID)
|
2016-08-22 20:57:27 +00:00
|
|
|
if err != nil {
|
2017-02-28 00:00:19 +00:00
|
|
|
return fmt.Errorf("failed to lookup allocation %q: %v", va.AllocID, err)
|
2016-08-22 20:57:27 +00:00
|
|
|
}
|
|
|
|
if alloc == nil || alloc.Terminated() {
|
|
|
|
// No longer running and should be revoked
|
|
|
|
revoke = append(revoke, va)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
// Check the node
|
2017-02-08 04:31:23 +00:00
|
|
|
node, err := state.NodeByID(ws, va.NodeID)
|
2016-08-22 20:57:27 +00:00
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("failed to lookup node %q: %v", va.NodeID, err)
|
|
|
|
}
|
|
|
|
if node == nil || node.TerminalStatus() {
|
|
|
|
// Node is terminal so any accessor from it should be revoked
|
|
|
|
revoke = append(revoke, va)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(revoke) != 0 {
|
|
|
|
if err := s.vault.RevokeTokens(context.Background(), revoke, true); err != nil {
|
|
|
|
return fmt.Errorf("failed to revoke tokens: %v", err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-12-06 20:46:46 +00:00
|
|
|
// revokeSITokenAccessorsOnRestore is used to revoke Service Identity token
|
|
|
|
// accessors on behalf of allocs that are now gone / terminal.
|
|
|
|
func (s *Server) revokeSITokenAccessorsOnRestore() error {
|
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
fsmState := s.fsm.State()
|
|
|
|
iter, err := fsmState.SITokenAccessors(ws)
|
|
|
|
if err != nil {
|
|
|
|
return errors.Wrap(err, "failed to get SI token accessors")
|
|
|
|
}
|
|
|
|
|
|
|
|
var toRevoke []*structs.SITokenAccessor
|
|
|
|
for raw := iter.Next(); raw != nil; raw = iter.Next() {
|
|
|
|
accessor := raw.(*structs.SITokenAccessor)
|
|
|
|
|
|
|
|
// Check the allocation
|
|
|
|
alloc, err := fsmState.AllocByID(ws, accessor.AllocID)
|
|
|
|
if err != nil {
|
|
|
|
return errors.Wrapf(err, "failed to lookup alloc %q", accessor.AllocID)
|
|
|
|
}
|
|
|
|
if alloc == nil || alloc.Terminated() {
|
|
|
|
// no longer running and associated accessors should be revoked
|
|
|
|
toRevoke = append(toRevoke, accessor)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
// Check the node
|
|
|
|
node, err := fsmState.NodeByID(ws, accessor.NodeID)
|
|
|
|
if err != nil {
|
|
|
|
return errors.Wrapf(err, "failed to lookup node %q", accessor.NodeID)
|
|
|
|
}
|
|
|
|
if node == nil || node.TerminalStatus() {
|
|
|
|
// node is terminal and associated accessors should be revoked
|
|
|
|
toRevoke = append(toRevoke, accessor)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(toRevoke) > 0 {
|
|
|
|
ctx := context.Background()
|
2020-01-15 15:29:47 +00:00
|
|
|
s.consulACLs.RevokeTokens(ctx, toRevoke, true)
|
2019-12-06 20:46:46 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-12-04 23:10:08 +00:00
|
|
|
// restorePeriodicDispatcher is used to restore all periodic jobs into the
|
|
|
|
// periodic dispatcher. It also determines if a periodic job should have been
|
|
|
|
// created during the leadership transition and force runs them. The periodic
|
|
|
|
// dispatcher is maintained only by the leader, so it must be restored anytime a
|
|
|
|
// leadership transition takes place.
|
2015-12-18 20:26:28 +00:00
|
|
|
func (s *Server) restorePeriodicDispatcher() error {
|
2018-09-15 23:23:13 +00:00
|
|
|
logger := s.logger.Named("periodic")
|
2017-02-08 04:31:23 +00:00
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
iter, err := s.fsm.State().JobsByPeriodic(ws, true)
|
2015-12-04 23:10:08 +00:00
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("failed to get periodic jobs: %v", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
now := time.Now()
|
|
|
|
for i := iter.Next(); i != nil; i = iter.Next() {
|
|
|
|
job := i.(*structs.Job)
|
2017-08-03 19:37:58 +00:00
|
|
|
|
|
|
|
// We skip adding parameterized jobs because they themselves aren't
|
|
|
|
// tracked, only the dispatched children are.
|
|
|
|
if job.IsParameterized() {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2017-12-11 21:55:17 +00:00
|
|
|
if err := s.periodicDispatcher.Add(job); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
logger.Error("failed to add job to periodic dispatcher", "error", err)
|
2018-04-26 22:51:47 +00:00
|
|
|
continue
|
2017-09-12 21:25:40 +00:00
|
|
|
}
|
|
|
|
|
2017-12-11 21:55:17 +00:00
|
|
|
// We do not need to force run the job since it isn't active.
|
|
|
|
if !job.IsPeriodicActive() {
|
2017-09-12 21:25:40 +00:00
|
|
|
continue
|
|
|
|
}
|
2015-12-04 23:10:08 +00:00
|
|
|
|
2015-12-16 22:14:55 +00:00
|
|
|
// If the periodic job has never been launched before, launch will hold
|
|
|
|
// the time the periodic job was added. Otherwise it has the last launch
|
|
|
|
// time of the periodic job.
|
2017-09-07 23:56:15 +00:00
|
|
|
launch, err := s.fsm.State().PeriodicLaunchByID(ws, job.Namespace, job.ID)
|
2017-12-11 21:55:17 +00:00
|
|
|
if err != nil {
|
2015-12-19 01:51:30 +00:00
|
|
|
return fmt.Errorf("failed to get periodic launch time: %v", err)
|
2015-12-04 23:10:08 +00:00
|
|
|
}
|
2017-12-11 21:55:17 +00:00
|
|
|
if launch == nil {
|
|
|
|
return fmt.Errorf("no recorded periodic launch time for job %q in namespace %q",
|
|
|
|
job.ID, job.Namespace)
|
|
|
|
}
|
2015-12-04 23:10:08 +00:00
|
|
|
|
2015-12-16 22:14:55 +00:00
|
|
|
// nextLaunch is the next launch that should occur.
|
2018-04-26 20:57:45 +00:00
|
|
|
nextLaunch, err := job.Periodic.Next(launch.Launch.In(job.Periodic.GetLocation()))
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
logger.Error("failed to determine next periodic launch for job", "job", job.NamespacedID(), "error", err)
|
2018-04-26 22:15:43 +00:00
|
|
|
continue
|
2018-04-26 20:57:45 +00:00
|
|
|
}
|
2015-12-16 22:14:55 +00:00
|
|
|
|
|
|
|
// We skip force launching the job if there should be no next launch
|
|
|
|
// (the zero case) or if the next launch time is in the future. If it is
|
|
|
|
// in the future, it will be handled by the periodic dispatcher.
|
|
|
|
if nextLaunch.IsZero() || !nextLaunch.Before(now) {
|
2015-12-04 23:10:08 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2017-09-07 23:56:15 +00:00
|
|
|
if _, err := s.periodicDispatcher.ForceRun(job.Namespace, job.ID); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
logger.Error("force run of periodic job failed", "job", job.NamespacedID(), "error", err)
|
|
|
|
return fmt.Errorf("force run of periodic job %q failed: %v", job.NamespacedID(), err)
|
2015-12-04 23:10:08 +00:00
|
|
|
}
|
2018-09-15 23:23:13 +00:00
|
|
|
logger.Debug("periodic job force runned during leadership establishment", "job", job.NamespacedID())
|
2015-12-04 23:10:08 +00:00
|
|
|
}
|
|
|
|
|
2015-12-18 20:26:28 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-08-15 22:15:00 +00:00
|
|
|
// schedulePeriodic is used to do periodic job dispatch while we are leader
|
|
|
|
func (s *Server) schedulePeriodic(stopCh chan struct{}) {
|
|
|
|
evalGC := time.NewTicker(s.config.EvalGCInterval)
|
|
|
|
defer evalGC.Stop()
|
2015-09-07 18:01:29 +00:00
|
|
|
nodeGC := time.NewTicker(s.config.NodeGCInterval)
|
|
|
|
defer nodeGC.Stop()
|
2015-12-15 03:20:57 +00:00
|
|
|
jobGC := time.NewTicker(s.config.JobGCInterval)
|
|
|
|
defer jobGC.Stop()
|
2017-09-25 18:03:20 +00:00
|
|
|
deploymentGC := time.NewTicker(s.config.DeploymentGCInterval)
|
|
|
|
defer deploymentGC.Stop()
|
2015-08-15 22:15:00 +00:00
|
|
|
|
2016-06-22 16:11:25 +00:00
|
|
|
// getLatest grabs the latest index from the state store. It returns true if
|
|
|
|
// the index was retrieved successfully.
|
|
|
|
getLatest := func() (uint64, bool) {
|
2016-06-22 16:33:15 +00:00
|
|
|
snapshotIndex, err := s.fsm.State().LatestIndex()
|
2016-06-22 16:04:22 +00:00
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to determine state store's index", "error", err)
|
2016-06-22 16:11:25 +00:00
|
|
|
return 0, false
|
2016-06-22 16:04:22 +00:00
|
|
|
}
|
|
|
|
|
2016-06-22 16:11:25 +00:00
|
|
|
return snapshotIndex, true
|
|
|
|
}
|
|
|
|
|
|
|
|
for {
|
|
|
|
|
2015-08-15 22:15:00 +00:00
|
|
|
select {
|
|
|
|
case <-evalGC.C:
|
2016-06-22 16:11:25 +00:00
|
|
|
if index, ok := getLatest(); ok {
|
|
|
|
s.evalBroker.Enqueue(s.coreJobEval(structs.CoreJobEvalGC, index))
|
|
|
|
}
|
2015-09-07 18:01:29 +00:00
|
|
|
case <-nodeGC.C:
|
2016-06-22 16:11:25 +00:00
|
|
|
if index, ok := getLatest(); ok {
|
|
|
|
s.evalBroker.Enqueue(s.coreJobEval(structs.CoreJobNodeGC, index))
|
|
|
|
}
|
2015-12-15 03:20:57 +00:00
|
|
|
case <-jobGC.C:
|
2016-06-22 16:11:25 +00:00
|
|
|
if index, ok := getLatest(); ok {
|
|
|
|
s.evalBroker.Enqueue(s.coreJobEval(structs.CoreJobJobGC, index))
|
|
|
|
}
|
2017-09-25 18:03:20 +00:00
|
|
|
case <-deploymentGC.C:
|
|
|
|
if index, ok := getLatest(); ok {
|
|
|
|
s.evalBroker.Enqueue(s.coreJobEval(structs.CoreJobDeploymentGC, index))
|
|
|
|
}
|
2015-08-15 22:15:00 +00:00
|
|
|
case <-stopCh:
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-08-15 23:07:50 +00:00
|
|
|
// coreJobEval returns an evaluation for a core job
|
2016-06-22 16:04:22 +00:00
|
|
|
func (s *Server) coreJobEval(job string, modifyIndex uint64) *structs.Evaluation {
|
2015-08-15 23:07:50 +00:00
|
|
|
return &structs.Evaluation{
|
2017-09-29 16:58:48 +00:00
|
|
|
ID: uuid.Generate(),
|
2017-09-07 23:56:15 +00:00
|
|
|
Namespace: "-",
|
2015-08-15 22:15:00 +00:00
|
|
|
Priority: structs.CoreJobPriority,
|
|
|
|
Type: structs.JobTypeCore,
|
|
|
|
TriggeredBy: structs.EvalTriggerScheduled,
|
|
|
|
JobID: job,
|
2017-10-23 22:04:00 +00:00
|
|
|
LeaderACL: s.getLeaderAcl(),
|
2015-08-15 22:15:00 +00:00
|
|
|
Status: structs.EvalStatusPending,
|
2016-06-22 16:04:22 +00:00
|
|
|
ModifyIndex: modifyIndex,
|
2015-08-15 22:15:00 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-08-16 18:10:18 +00:00
|
|
|
// reapFailedEvaluations is used to reap evaluations that
|
|
|
|
// have reached their delivery limit and should be failed
|
|
|
|
func (s *Server) reapFailedEvaluations(stopCh chan struct{}) {
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-stopCh:
|
|
|
|
return
|
|
|
|
default:
|
|
|
|
// Scan for a failed evaluation
|
|
|
|
eval, token, err := s.evalBroker.Dequeue([]string{failedQueue}, time.Second)
|
|
|
|
if err != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if eval == nil {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
// Update the status to failed
|
2017-04-14 20:19:14 +00:00
|
|
|
updateEval := eval.Copy()
|
|
|
|
updateEval.Status = structs.EvalStatusFailed
|
|
|
|
updateEval.StatusDescription = fmt.Sprintf("evaluation reached delivery limit (%d)", s.config.EvalDeliveryLimit)
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Warn("eval reached delivery limit, marking as failed", "eval", updateEval.GoString())
|
2015-08-16 18:10:18 +00:00
|
|
|
|
2017-04-12 21:47:59 +00:00
|
|
|
// Create a follow-up evaluation that will be used to retry the
|
|
|
|
// scheduling for the job after the cluster is hopefully more stable
|
|
|
|
// due to the fairly large backoff.
|
2017-04-14 22:24:55 +00:00
|
|
|
followupEvalWait := s.config.EvalFailedFollowupBaselineDelay +
|
|
|
|
time.Duration(rand.Int63n(int64(s.config.EvalFailedFollowupDelayRange)))
|
2019-08-07 16:50:35 +00:00
|
|
|
|
2017-04-12 21:47:59 +00:00
|
|
|
followupEval := eval.CreateFailedFollowUpEval(followupEvalWait)
|
2019-02-21 00:07:11 +00:00
|
|
|
updateEval.NextEval = followupEval.ID
|
2019-08-07 16:50:35 +00:00
|
|
|
updateEval.UpdateModifyTime()
|
2017-04-12 21:47:59 +00:00
|
|
|
|
2015-08-16 18:10:18 +00:00
|
|
|
// Update via Raft
|
|
|
|
req := structs.EvalUpdateRequest{
|
2017-04-14 20:19:14 +00:00
|
|
|
Evals: []*structs.Evaluation{updateEval, followupEval},
|
2015-08-16 18:10:18 +00:00
|
|
|
}
|
|
|
|
if _, _, err := s.raftApply(structs.EvalUpdateRequestType, &req); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to update failed eval and create a follow-up", "eval", updateEval.GoString(), "error", err)
|
2015-08-16 18:10:18 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
// Ack completion
|
|
|
|
s.evalBroker.Ack(eval.ID, token)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-01-31 00:16:13 +00:00
|
|
|
// reapDupBlockedEvaluations is used to reap duplicate blocked evaluations and
|
|
|
|
// should be cancelled.
|
|
|
|
func (s *Server) reapDupBlockedEvaluations(stopCh chan struct{}) {
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-stopCh:
|
|
|
|
return
|
|
|
|
default:
|
|
|
|
// Scan for duplicate blocked evals.
|
|
|
|
dups := s.blockedEvals.GetDuplicates(time.Second)
|
|
|
|
if dups == nil {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
cancel := make([]*structs.Evaluation, len(dups))
|
|
|
|
for i, dup := range dups {
|
|
|
|
// Update the status to cancelled
|
|
|
|
newEval := dup.Copy()
|
|
|
|
newEval.Status = structs.EvalStatusCancelled
|
|
|
|
newEval.StatusDescription = fmt.Sprintf("existing blocked evaluation exists for job %q", newEval.JobID)
|
2019-08-07 16:50:35 +00:00
|
|
|
newEval.UpdateModifyTime()
|
2016-01-31 00:16:13 +00:00
|
|
|
cancel[i] = newEval
|
|
|
|
}
|
|
|
|
|
|
|
|
// Update via Raft
|
|
|
|
req := structs.EvalUpdateRequest{
|
|
|
|
Evals: cancel,
|
|
|
|
}
|
|
|
|
if _, _, err := s.raftApply(structs.EvalUpdateRequestType, &req); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to update duplicate evals", "evals", log.Fmt("%#v", cancel), "error", err)
|
2016-01-31 00:16:13 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-05-23 23:27:26 +00:00
|
|
|
// periodicUnblockFailedEvals periodically unblocks failed, blocked evaluations.
|
|
|
|
func (s *Server) periodicUnblockFailedEvals(stopCh chan struct{}) {
|
2016-07-06 00:08:58 +00:00
|
|
|
ticker := time.NewTicker(failedEvalUnblockInterval)
|
2016-05-25 17:28:25 +00:00
|
|
|
defer ticker.Stop()
|
2016-05-23 23:27:26 +00:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-stopCh:
|
|
|
|
return
|
|
|
|
case <-ticker.C:
|
|
|
|
// Unblock the failed allocations
|
|
|
|
s.blockedEvals.UnblockFailed()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-10-30 19:19:11 +00:00
|
|
|
// publishJobSummaryMetrics publishes the job summaries as metrics
|
|
|
|
func (s *Server) publishJobSummaryMetrics(stopCh chan struct{}) {
|
|
|
|
timer := time.NewTimer(0)
|
|
|
|
defer timer.Stop()
|
|
|
|
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-stopCh:
|
|
|
|
return
|
|
|
|
case <-timer.C:
|
2017-11-01 20:14:44 +00:00
|
|
|
timer.Reset(s.config.StatsCollectionInterval)
|
2017-10-30 19:19:11 +00:00
|
|
|
state, err := s.State().Snapshot()
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to get state", "error", err)
|
2017-10-30 19:19:11 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
iter, err := state.JobSummaries(ws)
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to get job summaries", "error", err)
|
2017-10-30 19:19:11 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
for {
|
|
|
|
raw := iter.Next()
|
|
|
|
if raw == nil {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
summary := raw.(*structs.JobSummary)
|
2018-11-14 21:04:03 +00:00
|
|
|
if s.config.DisableDispatchedJobSummaryMetrics {
|
|
|
|
job, err := state.JobByID(ws, summary.Namespace, summary.JobID)
|
|
|
|
if err != nil {
|
2018-11-30 03:27:39 +00:00
|
|
|
s.logger.Error("error getting job for summary", "error", err)
|
2018-11-14 21:04:03 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
if job.Dispatched {
|
|
|
|
continue
|
|
|
|
}
|
2017-10-30 19:19:11 +00:00
|
|
|
}
|
2018-11-14 16:13:52 +00:00
|
|
|
s.iterateJobSummaryMetrics(summary)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) iterateJobSummaryMetrics(summary *structs.JobSummary) {
|
|
|
|
for name, tgSummary := range summary.Summary {
|
|
|
|
if !s.config.DisableTaggedMetrics {
|
|
|
|
labels := []metrics.Label{
|
|
|
|
{
|
|
|
|
Name: "job",
|
|
|
|
Value: summary.JobID,
|
|
|
|
},
|
|
|
|
{
|
|
|
|
Name: "task_group",
|
|
|
|
Value: name,
|
|
|
|
},
|
2019-06-17 16:52:49 +00:00
|
|
|
{
|
|
|
|
Name: "namespace",
|
|
|
|
Value: summary.Namespace,
|
|
|
|
},
|
2017-10-30 19:19:11 +00:00
|
|
|
}
|
2018-11-14 16:13:52 +00:00
|
|
|
|
|
|
|
if strings.Contains(summary.JobID, "/dispatch-") {
|
|
|
|
jobInfo := strings.Split(summary.JobID, "/dispatch-")
|
|
|
|
labels = append(labels, metrics.Label{
|
|
|
|
Name: "parent_id",
|
|
|
|
Value: jobInfo[0],
|
|
|
|
}, metrics.Label{
|
|
|
|
Name: "dispatch_id",
|
|
|
|
Value: jobInfo[1],
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
if strings.Contains(summary.JobID, "/periodic-") {
|
|
|
|
jobInfo := strings.Split(summary.JobID, "/periodic-")
|
|
|
|
labels = append(labels, metrics.Label{
|
|
|
|
Name: "parent_id",
|
|
|
|
Value: jobInfo[0],
|
|
|
|
}, metrics.Label{
|
|
|
|
Name: "periodic_id",
|
|
|
|
Value: jobInfo[1],
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
metrics.SetGaugeWithLabels([]string{"nomad", "job_summary", "queued"},
|
|
|
|
float32(tgSummary.Queued), labels)
|
|
|
|
metrics.SetGaugeWithLabels([]string{"nomad", "job_summary", "complete"},
|
|
|
|
float32(tgSummary.Complete), labels)
|
|
|
|
metrics.SetGaugeWithLabels([]string{"nomad", "job_summary", "failed"},
|
|
|
|
float32(tgSummary.Failed), labels)
|
|
|
|
metrics.SetGaugeWithLabels([]string{"nomad", "job_summary", "running"},
|
|
|
|
float32(tgSummary.Running), labels)
|
|
|
|
metrics.SetGaugeWithLabels([]string{"nomad", "job_summary", "starting"},
|
|
|
|
float32(tgSummary.Starting), labels)
|
|
|
|
metrics.SetGaugeWithLabels([]string{"nomad", "job_summary", "lost"},
|
|
|
|
float32(tgSummary.Lost), labels)
|
|
|
|
}
|
|
|
|
if s.config.BackwardsCompatibleMetrics {
|
|
|
|
metrics.SetGauge([]string{"nomad", "job_summary", summary.JobID, name, "queued"}, float32(tgSummary.Queued))
|
|
|
|
metrics.SetGauge([]string{"nomad", "job_summary", summary.JobID, name, "complete"}, float32(tgSummary.Complete))
|
|
|
|
metrics.SetGauge([]string{"nomad", "job_summary", summary.JobID, name, "failed"}, float32(tgSummary.Failed))
|
|
|
|
metrics.SetGauge([]string{"nomad", "job_summary", summary.JobID, name, "running"}, float32(tgSummary.Running))
|
|
|
|
metrics.SetGauge([]string{"nomad", "job_summary", summary.JobID, name, "starting"}, float32(tgSummary.Starting))
|
|
|
|
metrics.SetGauge([]string{"nomad", "job_summary", summary.JobID, name, "lost"}, float32(tgSummary.Lost))
|
2017-10-30 19:19:11 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-07-24 13:17:33 +00:00
|
|
|
// publishJobStatusMetrics publishes the job statuses as metrics
|
|
|
|
func (s *Server) publishJobStatusMetrics(stopCh chan struct{}) {
|
|
|
|
timer := time.NewTimer(0)
|
|
|
|
defer timer.Stop()
|
|
|
|
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-stopCh:
|
|
|
|
return
|
|
|
|
case <-timer.C:
|
|
|
|
timer.Reset(s.config.StatsCollectionInterval)
|
|
|
|
state, err := s.State().Snapshot()
|
|
|
|
if err != nil {
|
|
|
|
s.logger.Error("failed to get state", "error", err)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
ws := memdb.NewWatchSet()
|
|
|
|
iter, err := state.Jobs(ws)
|
|
|
|
if err != nil {
|
|
|
|
s.logger.Error("failed to get job statuses", "error", err)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
s.iterateJobStatusMetrics(&iter)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) iterateJobStatusMetrics(jobs *memdb.ResultIterator) {
|
|
|
|
var pending int64 // Sum of all jobs in 'pending' state
|
|
|
|
var running int64 // Sum of all jobs in 'running' state
|
|
|
|
var dead int64 // Sum of all jobs in 'dead' state
|
|
|
|
|
|
|
|
for {
|
|
|
|
raw := (*jobs).Next()
|
|
|
|
if raw == nil {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
job := raw.(*structs.Job)
|
|
|
|
|
|
|
|
switch job.Status {
|
|
|
|
case structs.JobStatusPending:
|
|
|
|
pending++
|
|
|
|
case structs.JobStatusRunning:
|
|
|
|
running++
|
|
|
|
case structs.JobStatusDead:
|
|
|
|
dead++
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
metrics.SetGauge([]string{"nomad", "job_status", "pending"}, float32(pending))
|
|
|
|
metrics.SetGauge([]string{"nomad", "job_status", "running"}, float32(running))
|
|
|
|
metrics.SetGauge([]string{"nomad", "job_status", "dead"}, float32(dead))
|
|
|
|
}
|
|
|
|
|
2015-07-24 04:58:38 +00:00
|
|
|
// revokeLeadership is invoked once we step down as leader.
|
|
|
|
// This is used to cleanup any state that may be specific to a leader.
|
|
|
|
func (s *Server) revokeLeadership() error {
|
2018-02-20 18:23:11 +00:00
|
|
|
defer metrics.MeasureSince([]string{"nomad", "leader", "revoke_leadership"}, time.Now())
|
2018-02-20 18:22:15 +00:00
|
|
|
|
2019-07-02 07:58:02 +00:00
|
|
|
s.resetConsistentReadReady()
|
|
|
|
|
2017-10-23 22:11:13 +00:00
|
|
|
// Clear the leader token since we are no longer the leader.
|
|
|
|
s.setLeaderAcl("")
|
|
|
|
|
2017-12-18 21:16:23 +00:00
|
|
|
// Disable autopilot
|
|
|
|
s.autopilot.Stop()
|
|
|
|
|
2015-07-27 22:11:42 +00:00
|
|
|
// Disable the plan queue, since we are no longer leader
|
|
|
|
s.planQueue.SetEnabled(false)
|
|
|
|
|
2015-07-24 04:58:38 +00:00
|
|
|
// Disable the eval broker, since it is only useful as a leader
|
|
|
|
s.evalBroker.SetEnabled(false)
|
2015-08-23 00:17:13 +00:00
|
|
|
|
2016-01-31 00:21:37 +00:00
|
|
|
// Disable the blocked eval tracker, since it is only useful as a leader
|
|
|
|
s.blockedEvals.SetEnabled(false)
|
|
|
|
|
2015-12-18 20:26:28 +00:00
|
|
|
// Disable the periodic dispatcher, since it is only useful as a leader
|
|
|
|
s.periodicDispatcher.SetEnabled(false)
|
|
|
|
|
2016-08-22 20:57:27 +00:00
|
|
|
// Disable the Vault client as it is only useful as a leader.
|
|
|
|
s.vault.SetActive(false)
|
|
|
|
|
2017-06-28 22:35:52 +00:00
|
|
|
// Disable the deployment watcher as it is only useful as a leader.
|
2018-02-27 00:28:10 +00:00
|
|
|
s.deploymentWatcher.SetEnabled(false, nil)
|
|
|
|
|
|
|
|
// Disable the node drainer
|
|
|
|
s.nodeDrainer.SetEnabled(false, nil)
|
2017-06-28 22:35:52 +00:00
|
|
|
|
2017-09-07 23:56:15 +00:00
|
|
|
// Disable any enterprise systems required.
|
|
|
|
if err := s.revokeEnterpriseLeadership(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2015-08-23 00:17:13 +00:00
|
|
|
// Clear the heartbeat timers on either shutdown or step down,
|
|
|
|
// since we are no longer responsible for TTL expirations.
|
|
|
|
if err := s.clearAllHeartbeatTimers(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("clearing heartbeat timers failed", "error", err)
|
2015-08-23 00:17:13 +00:00
|
|
|
return err
|
|
|
|
}
|
2015-08-23 20:59:13 +00:00
|
|
|
|
|
|
|
// Unpause our worker if we paused previously
|
|
|
|
if len(s.workers) > 1 {
|
2016-02-19 22:49:43 +00:00
|
|
|
for i := 0; i < len(s.workers)/2; i++ {
|
2016-02-17 21:50:06 +00:00
|
|
|
s.workers[i].SetPause(false)
|
|
|
|
}
|
2015-08-23 20:59:13 +00:00
|
|
|
}
|
2015-07-24 04:58:38 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-06-05 21:54:45 +00:00
|
|
|
// reconcile is used to reconcile the differences between Serf
|
|
|
|
// membership and what is reflected in our strongly consistent store.
|
|
|
|
func (s *Server) reconcile() error {
|
|
|
|
defer metrics.MeasureSince([]string{"nomad", "leader", "reconcile"}, time.Now())
|
|
|
|
members := s.serf.Members()
|
|
|
|
for _, member := range members {
|
|
|
|
if err := s.reconcileMember(member); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-06-04 11:38:41 +00:00
|
|
|
// reconcileMember is used to do an async reconcile of a single serf member
|
|
|
|
func (s *Server) reconcileMember(member serf.Member) error {
|
|
|
|
// Check if this is a member we should handle
|
|
|
|
valid, parts := isNomadServer(member)
|
|
|
|
if !valid || parts.Region != s.config.Region {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
defer metrics.MeasureSince([]string{"nomad", "leader", "reconcileMember"}, time.Now())
|
|
|
|
|
|
|
|
var err error
|
|
|
|
switch member.Status {
|
|
|
|
case serf.StatusAlive:
|
|
|
|
err = s.addRaftPeer(member, parts)
|
|
|
|
case serf.StatusLeft, StatusReap:
|
|
|
|
err = s.removeRaftPeer(member, parts)
|
|
|
|
}
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to reconcile member", "member", member, "error", err)
|
2015-06-04 11:38:41 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// addRaftPeer is used to add a new Raft peer when a Nomad server joins
|
|
|
|
func (s *Server) addRaftPeer(m serf.Member, parts *serverParts) error {
|
|
|
|
// Check for possibility of multiple bootstrap nodes
|
2017-11-22 00:29:11 +00:00
|
|
|
members := s.serf.Members()
|
2015-06-04 11:38:41 +00:00
|
|
|
if parts.Bootstrap {
|
|
|
|
for _, member := range members {
|
|
|
|
valid, p := isNomadServer(member)
|
|
|
|
if valid && member.Name != m.Name && p.Bootstrap {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("skipping adding Raft peer because an existing peer is in bootstrap mode and only one server should be in bootstrap mode",
|
|
|
|
"existing_peer", member.Name, "joining_peer", m.Name)
|
2015-06-04 11:38:41 +00:00
|
|
|
return nil
|
|
|
|
}
|
2015-06-01 15:49:10 +00:00
|
|
|
}
|
|
|
|
}
|
2015-06-04 11:38:41 +00:00
|
|
|
|
2018-05-30 16:34:45 +00:00
|
|
|
// 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.
|
2017-11-22 00:29:11 +00:00
|
|
|
addr := (&net.TCPAddr{IP: m.Addr, Port: parts.Port}).String()
|
2017-02-02 23:49:06 +00:00
|
|
|
configFuture := s.raft.GetConfiguration()
|
|
|
|
if err := configFuture.Error(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to get raft configuration", "error", err)
|
2017-02-02 23:49:06 +00:00
|
|
|
return err
|
|
|
|
}
|
2018-05-30 16:34:45 +00:00
|
|
|
|
|
|
|
if m.Name == s.config.NodeName {
|
|
|
|
if l := len(configFuture.Configuration().Servers); l < 3 {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Debug("skipping self join check for peer since the cluster is too small", "peer", m.Name)
|
2017-02-02 23:49:06 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-11-22 00:29:11 +00:00
|
|
|
// 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.
|
2017-12-18 21:16:23 +00:00
|
|
|
minRaftProtocol, err := s.autopilot.MinRaftProtocol()
|
2017-11-22 00:29:11 +00:00
|
|
|
if err != nil {
|
2015-06-04 11:38:41 +00:00
|
|
|
return err
|
|
|
|
}
|
2017-11-22 00:29:11 +00:00
|
|
|
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) {
|
2018-05-30 16:34:45 +00:00
|
|
|
// Exit with no-op if this is being called on an existing server and both the ID and address match
|
2017-11-22 00:29:11 +00:00
|
|
|
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)
|
|
|
|
}
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Info("removed server with duplicate address", "address", server.Address)
|
2017-11-22 00:29:11 +00:00
|
|
|
} else {
|
|
|
|
if err := future.Error(); err != nil {
|
|
|
|
return fmt.Errorf("error removing server with duplicate ID %q: %s", server.ID, err)
|
|
|
|
}
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Info("removed server with duplicate ID", "id", server.ID)
|
2017-11-22 00:29:11 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Attempt to add as a peer
|
|
|
|
switch {
|
|
|
|
case minRaftProtocol >= 3:
|
2017-12-18 21:16:23 +00:00
|
|
|
addFuture := s.raft.AddNonvoter(raft.ServerID(parts.ID), raft.ServerAddress(addr), 0, 0)
|
2017-11-22 00:29:11 +00:00
|
|
|
if err := addFuture.Error(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to add raft peer", "error", err)
|
2017-11-22 00:29:11 +00:00
|
|
|
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 {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to add raft peer", "error", err)
|
2017-11-22 00:29:11 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
default:
|
|
|
|
addFuture := s.raft.AddPeer(raft.ServerAddress(addr))
|
|
|
|
if err := addFuture.Error(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to add raft peer", "error", err)
|
2017-11-22 00:29:11 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-06-04 11:38:41 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// removeRaftPeer is used to remove a Raft peer when a Nomad server leaves
|
|
|
|
// or is reaped
|
|
|
|
func (s *Server) removeRaftPeer(m serf.Member, parts *serverParts) error {
|
2017-02-02 23:49:06 +00:00
|
|
|
addr := (&net.TCPAddr{IP: m.Addr, Port: parts.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 {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to get raft configuration", "error", err)
|
2017-02-02 23:49:06 +00:00
|
|
|
return err
|
|
|
|
}
|
2017-11-22 00:29:11 +00:00
|
|
|
|
2017-12-18 21:16:23 +00:00
|
|
|
minRaftProtocol, err := s.autopilot.MinRaftProtocol()
|
2017-11-22 00:29:11 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Pick which remove API to use based on how the server was added.
|
2017-02-02 23:49:06 +00:00
|
|
|
for _, server := range configFuture.Configuration().Servers {
|
2017-11-22 00:29:11 +00:00
|
|
|
// 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) {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Info("removing server by ID", "id", server.ID)
|
2017-11-22 00:29:11 +00:00
|
|
|
future := s.raft.RemoveServer(raft.ServerID(parts.ID), 0, 0)
|
|
|
|
if err := future.Error(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to remove raft peer", "id", server.ID, "error", err)
|
2017-11-22 00:29:11 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
break
|
|
|
|
} else if server.Address == raft.ServerAddress(addr) {
|
|
|
|
// If not, use the old remove API
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Info("removing server by address", "address", server.Address)
|
2017-11-22 00:29:11 +00:00
|
|
|
future := s.raft.RemovePeer(raft.ServerAddress(addr))
|
|
|
|
if err := future.Error(); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to remove raft peer", "address", addr, "error", err)
|
2017-11-22 00:29:11 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
break
|
2017-02-02 23:49:06 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-06-04 11:38:41 +00:00
|
|
|
return nil
|
2015-06-01 15:49:10 +00:00
|
|
|
}
|
2017-08-13 23:16:59 +00:00
|
|
|
|
|
|
|
// replicateACLPolicies is used to replicate ACL policies from
|
|
|
|
// the authoritative region to this region.
|
|
|
|
func (s *Server) replicateACLPolicies(stopCh chan struct{}) {
|
2017-08-19 22:30:01 +00:00
|
|
|
req := structs.ACLPolicyListRequest{
|
|
|
|
QueryOptions: structs.QueryOptions{
|
2017-08-24 16:53:30 +00:00
|
|
|
Region: s.config.AuthoritativeRegion,
|
|
|
|
AllowStale: true,
|
2017-08-19 22:30:01 +00:00
|
|
|
},
|
|
|
|
}
|
2017-08-13 23:16:59 +00:00
|
|
|
limiter := rate.NewLimiter(replicationRateLimit, int(replicationRateLimit))
|
2018-09-15 23:42:38 +00:00
|
|
|
s.logger.Debug("starting ACL policy replication from authoritative region", "authoritative_region", req.Region)
|
2017-08-13 23:16:59 +00:00
|
|
|
|
|
|
|
START:
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-stopCh:
|
|
|
|
return
|
|
|
|
default:
|
|
|
|
// Rate limit how often we attempt replication
|
|
|
|
limiter.Wait(context.Background())
|
|
|
|
|
|
|
|
// Fetch the list of policies
|
|
|
|
var resp structs.ACLPolicyListResponse
|
2017-10-12 22:16:33 +00:00
|
|
|
req.AuthToken = s.ReplicationToken()
|
2017-08-13 23:16:59 +00:00
|
|
|
err := s.forwardRegion(s.config.AuthoritativeRegion,
|
|
|
|
"ACL.ListPolicies", &req, &resp)
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to fetch policies from authoritative region", "error", err)
|
2017-08-13 23:16:59 +00:00
|
|
|
goto ERR_WAIT
|
|
|
|
}
|
|
|
|
|
|
|
|
// Perform a two-way diff
|
|
|
|
delete, update := diffACLPolicies(s.State(), req.MinQueryIndex, resp.Policies)
|
|
|
|
|
|
|
|
// Delete policies that should not exist
|
|
|
|
if len(delete) > 0 {
|
|
|
|
args := &structs.ACLPolicyDeleteRequest{
|
|
|
|
Names: delete,
|
|
|
|
}
|
|
|
|
_, _, err := s.raftApply(structs.ACLPolicyDeleteRequestType, args)
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to delete policies", "error", err)
|
2017-08-13 23:16:59 +00:00
|
|
|
goto ERR_WAIT
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Fetch any outdated policies
|
|
|
|
var fetched []*structs.ACLPolicy
|
2017-08-20 22:30:18 +00:00
|
|
|
if len(update) > 0 {
|
|
|
|
req := structs.ACLPolicySetRequest{
|
|
|
|
Names: update,
|
|
|
|
QueryOptions: structs.QueryOptions{
|
2017-08-24 16:57:14 +00:00
|
|
|
Region: s.config.AuthoritativeRegion,
|
2017-10-12 22:16:33 +00:00
|
|
|
AuthToken: s.ReplicationToken(),
|
2017-08-24 16:57:14 +00:00
|
|
|
AllowStale: true,
|
|
|
|
MinQueryIndex: resp.Index - 1,
|
2017-08-20 22:30:18 +00:00
|
|
|
},
|
2017-08-13 23:16:59 +00:00
|
|
|
}
|
2017-08-20 22:30:18 +00:00
|
|
|
var reply structs.ACLPolicySetResponse
|
|
|
|
if err := s.forwardRegion(s.config.AuthoritativeRegion,
|
|
|
|
"ACL.GetPolicies", &req, &reply); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to fetch policies from authoritative region", "error", err)
|
2017-08-13 23:16:59 +00:00
|
|
|
goto ERR_WAIT
|
|
|
|
}
|
2017-08-20 22:30:18 +00:00
|
|
|
for _, policy := range reply.Policies {
|
|
|
|
fetched = append(fetched, policy)
|
2017-08-13 23:16:59 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Update local policies
|
|
|
|
if len(fetched) > 0 {
|
|
|
|
args := &structs.ACLPolicyUpsertRequest{
|
|
|
|
Policies: fetched,
|
|
|
|
}
|
|
|
|
_, _, err := s.raftApply(structs.ACLPolicyUpsertRequestType, args)
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to update policies", "error", err)
|
2017-08-13 23:16:59 +00:00
|
|
|
goto ERR_WAIT
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Update the minimum query index, blocks until there
|
|
|
|
// is a change.
|
|
|
|
req.MinQueryIndex = resp.Index
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ERR_WAIT:
|
|
|
|
select {
|
|
|
|
case <-time.After(s.config.ReplicationBackoff):
|
|
|
|
goto START
|
|
|
|
case <-stopCh:
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// diffACLPolicies is used to perform a two-way diff between the local
|
|
|
|
// policies and the remote policies to determine which policies need to
|
|
|
|
// be deleted or updated.
|
|
|
|
func diffACLPolicies(state *state.StateStore, minIndex uint64, remoteList []*structs.ACLPolicyListStub) (delete []string, update []string) {
|
|
|
|
// Construct a set of the local and remote policies
|
2017-08-30 17:06:56 +00:00
|
|
|
local := make(map[string][]byte)
|
2017-08-13 23:16:59 +00:00
|
|
|
remote := make(map[string]struct{})
|
|
|
|
|
|
|
|
// Add all the local policies
|
|
|
|
iter, err := state.ACLPolicies(nil)
|
|
|
|
if err != nil {
|
|
|
|
panic("failed to iterate local policies")
|
|
|
|
}
|
|
|
|
for {
|
|
|
|
raw := iter.Next()
|
|
|
|
if raw == nil {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
policy := raw.(*structs.ACLPolicy)
|
2017-08-30 17:06:56 +00:00
|
|
|
local[policy.Name] = policy.Hash
|
2017-08-13 23:16:59 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Iterate over the remote policies
|
|
|
|
for _, rp := range remoteList {
|
|
|
|
remote[rp.Name] = struct{}{}
|
|
|
|
|
|
|
|
// Check if the policy is missing locally
|
2017-08-30 17:06:56 +00:00
|
|
|
if localHash, ok := local[rp.Name]; !ok {
|
2017-08-13 23:16:59 +00:00
|
|
|
update = append(update, rp.Name)
|
|
|
|
|
2017-08-30 17:06:56 +00:00
|
|
|
// Check if policy is newer remotely and there is a hash mis-match.
|
|
|
|
} else if rp.ModifyIndex > minIndex && !bytes.Equal(localHash, rp.Hash) {
|
2017-08-13 23:16:59 +00:00
|
|
|
update = append(update, rp.Name)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Check if policy should be deleted
|
|
|
|
for lp := range local {
|
|
|
|
if _, ok := remote[lp]; !ok {
|
|
|
|
delete = append(delete, lp)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
// replicateACLTokens is used to replicate global ACL tokens from
|
|
|
|
// the authoritative region to this region.
|
|
|
|
func (s *Server) replicateACLTokens(stopCh chan struct{}) {
|
2017-08-13 23:45:13 +00:00
|
|
|
req := structs.ACLTokenListRequest{
|
|
|
|
GlobalOnly: true,
|
2017-08-19 22:30:01 +00:00
|
|
|
QueryOptions: structs.QueryOptions{
|
2017-08-24 16:53:30 +00:00
|
|
|
Region: s.config.AuthoritativeRegion,
|
|
|
|
AllowStale: true,
|
2017-08-19 22:30:01 +00:00
|
|
|
},
|
2017-08-13 23:45:13 +00:00
|
|
|
}
|
|
|
|
limiter := rate.NewLimiter(replicationRateLimit, int(replicationRateLimit))
|
2018-09-15 23:42:38 +00:00
|
|
|
s.logger.Debug("starting ACL token replication from authoritative region", "authoritative_region", req.Region)
|
2017-08-13 23:45:13 +00:00
|
|
|
|
|
|
|
START:
|
2017-08-13 23:16:59 +00:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-stopCh:
|
|
|
|
return
|
2017-08-13 23:45:13 +00:00
|
|
|
default:
|
|
|
|
// Rate limit how often we attempt replication
|
|
|
|
limiter.Wait(context.Background())
|
|
|
|
|
|
|
|
// Fetch the list of tokens
|
|
|
|
var resp structs.ACLTokenListResponse
|
2017-10-12 22:16:33 +00:00
|
|
|
req.AuthToken = s.ReplicationToken()
|
2017-08-13 23:45:13 +00:00
|
|
|
err := s.forwardRegion(s.config.AuthoritativeRegion,
|
|
|
|
"ACL.ListTokens", &req, &resp)
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to fetch tokens from authoritative region", "error", err)
|
2017-08-13 23:45:13 +00:00
|
|
|
goto ERR_WAIT
|
|
|
|
}
|
|
|
|
|
|
|
|
// Perform a two-way diff
|
|
|
|
delete, update := diffACLTokens(s.State(), req.MinQueryIndex, resp.Tokens)
|
|
|
|
|
|
|
|
// Delete tokens that should not exist
|
|
|
|
if len(delete) > 0 {
|
|
|
|
args := &structs.ACLTokenDeleteRequest{
|
|
|
|
AccessorIDs: delete,
|
|
|
|
}
|
|
|
|
_, _, err := s.raftApply(structs.ACLTokenDeleteRequestType, args)
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to delete tokens", "error", err)
|
2017-08-13 23:45:13 +00:00
|
|
|
goto ERR_WAIT
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-08-19 22:30:01 +00:00
|
|
|
// Fetch any outdated policies.
|
2017-08-13 23:45:13 +00:00
|
|
|
var fetched []*structs.ACLToken
|
2017-08-20 22:30:18 +00:00
|
|
|
if len(update) > 0 {
|
|
|
|
req := structs.ACLTokenSetRequest{
|
|
|
|
AccessorIDS: update,
|
|
|
|
QueryOptions: structs.QueryOptions{
|
2017-08-24 16:57:14 +00:00
|
|
|
Region: s.config.AuthoritativeRegion,
|
2017-10-12 22:16:33 +00:00
|
|
|
AuthToken: s.ReplicationToken(),
|
2017-08-24 16:57:14 +00:00
|
|
|
AllowStale: true,
|
|
|
|
MinQueryIndex: resp.Index - 1,
|
2017-08-20 22:30:18 +00:00
|
|
|
},
|
2017-08-13 23:45:13 +00:00
|
|
|
}
|
2017-08-20 22:30:18 +00:00
|
|
|
var reply structs.ACLTokenSetResponse
|
|
|
|
if err := s.forwardRegion(s.config.AuthoritativeRegion,
|
|
|
|
"ACL.GetTokens", &req, &reply); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to fetch tokens from authoritative region", "error", err)
|
2017-08-13 23:45:13 +00:00
|
|
|
goto ERR_WAIT
|
|
|
|
}
|
2017-08-20 22:30:18 +00:00
|
|
|
for _, token := range reply.Tokens {
|
|
|
|
fetched = append(fetched, token)
|
2017-08-13 23:45:13 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-08-19 22:30:01 +00:00
|
|
|
// Update local tokens
|
2017-08-13 23:45:13 +00:00
|
|
|
if len(fetched) > 0 {
|
|
|
|
args := &structs.ACLTokenUpsertRequest{
|
|
|
|
Tokens: fetched,
|
|
|
|
}
|
|
|
|
_, _, err := s.raftApply(structs.ACLTokenUpsertRequestType, args)
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Error("failed to update tokens", "error", err)
|
2017-08-13 23:45:13 +00:00
|
|
|
goto ERR_WAIT
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Update the minimum query index, blocks until there
|
|
|
|
// is a change.
|
|
|
|
req.MinQueryIndex = resp.Index
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ERR_WAIT:
|
|
|
|
select {
|
|
|
|
case <-time.After(s.config.ReplicationBackoff):
|
|
|
|
goto START
|
|
|
|
case <-stopCh:
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// diffACLTokens is used to perform a two-way diff between the local
|
|
|
|
// tokens and the remote tokens to determine which tokens need to
|
|
|
|
// be deleted or updated.
|
|
|
|
func diffACLTokens(state *state.StateStore, minIndex uint64, remoteList []*structs.ACLTokenListStub) (delete []string, update []string) {
|
|
|
|
// Construct a set of the local and remote policies
|
2017-08-30 17:06:56 +00:00
|
|
|
local := make(map[string][]byte)
|
2017-08-13 23:45:13 +00:00
|
|
|
remote := make(map[string]struct{})
|
|
|
|
|
|
|
|
// Add all the local global tokens
|
|
|
|
iter, err := state.ACLTokensByGlobal(nil, true)
|
|
|
|
if err != nil {
|
|
|
|
panic("failed to iterate local tokens")
|
|
|
|
}
|
|
|
|
for {
|
|
|
|
raw := iter.Next()
|
|
|
|
if raw == nil {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
token := raw.(*structs.ACLToken)
|
2017-08-30 17:06:56 +00:00
|
|
|
local[token.AccessorID] = token.Hash
|
2017-08-13 23:45:13 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Iterate over the remote tokens
|
|
|
|
for _, rp := range remoteList {
|
|
|
|
remote[rp.AccessorID] = struct{}{}
|
|
|
|
|
|
|
|
// Check if the token is missing locally
|
2017-08-30 17:06:56 +00:00
|
|
|
if localHash, ok := local[rp.AccessorID]; !ok {
|
2017-08-13 23:45:13 +00:00
|
|
|
update = append(update, rp.AccessorID)
|
|
|
|
|
2017-08-30 17:06:56 +00:00
|
|
|
// Check if policy is newer remotely and there is a hash mis-match.
|
|
|
|
} else if rp.ModifyIndex > minIndex && !bytes.Equal(localHash, rp.Hash) {
|
2017-08-13 23:45:13 +00:00
|
|
|
update = append(update, rp.AccessorID)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Check if local token should be deleted
|
|
|
|
for lp := range local {
|
|
|
|
if _, ok := remote[lp]; !ok {
|
|
|
|
delete = append(delete, lp)
|
2017-08-13 23:16:59 +00:00
|
|
|
}
|
|
|
|
}
|
2017-08-13 23:45:13 +00:00
|
|
|
return
|
2017-08-13 23:16:59 +00:00
|
|
|
}
|
2017-12-18 21:16:23 +00:00
|
|
|
|
|
|
|
// getOrCreateAutopilotConfig is used to get the autopilot config, initializing it if necessary
|
2018-01-30 03:53:34 +00:00
|
|
|
func (s *Server) getOrCreateAutopilotConfig() *structs.AutopilotConfig {
|
2017-12-18 21:16:23 +00:00
|
|
|
state := s.fsm.State()
|
|
|
|
_, config, err := state.AutopilotConfig()
|
|
|
|
if err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Named("autopilot").Error("failed to get autopilot config", "error", err)
|
2017-12-18 21:16:23 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
if config != nil {
|
|
|
|
return config
|
|
|
|
}
|
|
|
|
|
2019-03-04 09:49:32 +00:00
|
|
|
if !ServersMeetMinimumVersion(s.Members(), minAutopilotVersion, false) {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Named("autopilot").Warn("can't initialize until all servers are above minimum version", "min_version", minAutopilotVersion)
|
2017-12-18 21:16:23 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
config = s.config.AutopilotConfig
|
|
|
|
req := structs.AutopilotSetConfigRequest{Config: *config}
|
|
|
|
if _, _, err = s.raftApply(structs.AutopilotRequestType, req); err != nil {
|
2018-09-15 23:23:13 +00:00
|
|
|
s.logger.Named("autopilot").Error("failed to initialize config", "error", err)
|
2017-12-18 21:16:23 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
return config
|
|
|
|
}
|
2018-09-28 04:27:38 +00:00
|
|
|
|
2018-11-01 22:05:17 +00:00
|
|
|
// getOrCreateSchedulerConfig is used to get the scheduler config. We create a default
|
|
|
|
// config if it doesn't already exist for bootstrapping an empty cluster
|
2018-09-28 04:27:38 +00:00
|
|
|
func (s *Server) getOrCreateSchedulerConfig() *structs.SchedulerConfiguration {
|
|
|
|
state := s.fsm.State()
|
|
|
|
_, config, err := state.SchedulerConfig()
|
|
|
|
if err != nil {
|
|
|
|
s.logger.Named("core").Error("failed to get scheduler config", "error", err)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
if config != nil {
|
|
|
|
return config
|
|
|
|
}
|
2019-03-04 09:49:32 +00:00
|
|
|
if !ServersMeetMinimumVersion(s.Members(), minSchedulerConfigVersion, false) {
|
2019-01-29 19:48:45 +00:00
|
|
|
s.logger.Named("core").Warn("can't initialize scheduler config until all servers are above minimum version", "min_version", minSchedulerConfigVersion)
|
2019-01-29 18:47:42 +00:00
|
|
|
return nil
|
|
|
|
}
|
2018-09-28 04:27:38 +00:00
|
|
|
|
2020-01-28 16:09:36 +00:00
|
|
|
req := structs.SchedulerSetConfigRequest{Config: s.config.DefaultSchedulerConfig}
|
2018-09-28 04:27:38 +00:00
|
|
|
if _, _, err = s.raftApply(structs.SchedulerConfigRequestType, req); err != nil {
|
|
|
|
s.logger.Named("core").Error("failed to initialize config", "error", err)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
return config
|
|
|
|
}
|
2019-11-14 13:18:29 +00:00
|
|
|
|
|
|
|
func (s *Server) generateClusterID() (string, error) {
|
|
|
|
if !ServersMeetMinimumVersion(s.Members(), minClusterIDVersion, false) {
|
|
|
|
s.logger.Named("core").Warn("cannot initialize cluster ID until all servers are above minimum version", "min_version", minClusterIDVersion)
|
|
|
|
return "", errors.Errorf("cluster ID cannot be created until all servers are above minimum version %s", minClusterIDVersion)
|
|
|
|
}
|
|
|
|
|
|
|
|
newMeta := structs.ClusterMetadata{ClusterID: uuid.Generate(), CreateTime: time.Now().UnixNano()}
|
|
|
|
if _, _, err := s.raftApply(structs.ClusterMetadataRequestType, newMeta); err != nil {
|
|
|
|
s.logger.Named("core").Error("failed to create cluster ID", "error", err)
|
|
|
|
return "", errors.Wrap(err, "failed to create cluster ID")
|
|
|
|
}
|
|
|
|
|
|
|
|
s.logger.Named("core").Info("established cluster id", "cluster_id", newMeta.ClusterID, "create_time", newMeta.CreateTime)
|
|
|
|
return newMeta.ClusterID, nil
|
|
|
|
}
|