Add metrics for blocked eval resources (#10454)

* add metrics for blocked eval resources

* docs: add new blocked_evals metrics

* fix to call `pruneStats` instead of `stats.prune` directly
This commit is contained in:
Luiz Aoqui
2021-04-29 15:03:45 -04:00
committed by GitHub
parent fc5056b83c
commit c7114921fa
11 changed files with 961 additions and 451 deletions

View File

@@ -374,6 +374,7 @@ type AllocationMetric struct {
ClassExhausted map[string]int
DimensionExhausted map[string]int
QuotaExhausted []string
ResourcesExhausted map[string]*Resources
// Deprecated, replaced with ScoreMetaData
Scores map[string]float64
AllocationTime time.Duration

View File

@@ -95,20 +95,6 @@ type wrappedEval struct {
token string
}
// BlockedStats returns all the stats about the blocked eval tracker.
type BlockedStats struct {
// TotalEscaped is the total number of blocked evaluations that have escaped
// computed node classes.
TotalEscaped int
// TotalBlocked is the total number of blocked evaluations.
TotalBlocked int
// TotalQuotaLimit is the total number of blocked evaluations that are due
// to the quota limit being reached.
TotalQuotaLimit int
}
// NewBlockedEvals creates a new blocked eval tracker that will enqueue
// unblocked evals into the passed broker.
func NewBlockedEvals(evalBroker *EvalBroker, logger log.Logger) *BlockedEvals {
@@ -123,7 +109,7 @@ func NewBlockedEvals(evalBroker *EvalBroker, logger log.Logger) *BlockedEvals {
capacityChangeCh: make(chan *capacityUpdate, unblockBuffer),
duplicateCh: make(chan struct{}, 1),
stopCh: make(chan struct{}),
stats: new(BlockedStats),
stats: NewBlockedStats(),
}
}
@@ -209,7 +195,7 @@ func (b *BlockedEvals) processBlock(eval *structs.Evaluation, token string) {
// Mark the job as tracked.
b.jobs[structs.NewNamespacedID(eval.JobID, eval.Namespace)] = eval.ID
b.stats.TotalBlocked++
b.stats.Block(eval)
// Track that the evaluation is being added due to reaching the quota limit
if eval.QuotaLimitReached != "" {
@@ -263,7 +249,7 @@ func (b *BlockedEvals) processBlockJobDuplicate(eval *structs.Evaluation) (newCa
if ok {
if latestEvalIndex(existingW.eval) <= latestEvalIndex(eval) {
delete(b.captured, existingID)
b.stats.TotalBlocked--
b.stats.Unblock(eval)
dup = existingW.eval
} else {
dup = eval
@@ -379,7 +365,7 @@ func (b *BlockedEvals) Untrack(jobID, namespace string) {
if evals, ok := b.system.JobEvals(nsID); ok {
for _, e := range evals {
b.system.Remove(e)
b.stats.TotalBlocked--
b.stats.Unblock(e)
}
return
}
@@ -395,7 +381,7 @@ func (b *BlockedEvals) Untrack(jobID, namespace string) {
if w, ok := b.captured[evalID]; ok {
delete(b.jobs, nsID)
delete(b.captured, evalID)
b.stats.TotalBlocked--
b.stats.Unblock(w.eval)
if w.eval.QuotaLimitReached != "" {
b.stats.TotalQuotaLimit--
}
@@ -405,7 +391,7 @@ func (b *BlockedEvals) Untrack(jobID, namespace string) {
delete(b.jobs, nsID)
delete(b.escaped, evalID)
b.stats.TotalEscaped--
b.stats.TotalBlocked--
b.stats.Unblock(w.eval)
if w.eval.QuotaLimitReached != "" {
b.stats.TotalQuotaLimit--
}
@@ -511,7 +497,7 @@ func (b *BlockedEvals) UnblockNode(nodeID string, index uint64) {
for e := range evals {
b.system.Remove(e)
b.stats.TotalBlocked--
b.stats.Unblock(e)
}
b.evalBroker.EnqueueAll(evals)
@@ -583,11 +569,13 @@ func (b *BlockedEvals) unblock(computedClass, quota string, index uint64) {
}
}
if l := len(unblocked); l != 0 {
if len(unblocked) != 0 {
// Update the counters
b.stats.TotalEscaped = 0
b.stats.TotalBlocked -= l
b.stats.TotalQuotaLimit -= numQuotaLimit
for eval := range unblocked {
b.stats.Unblock(eval)
}
// Enqueue all the unblocked evals into the broker.
b.evalBroker.EnqueueAll(unblocked)
@@ -630,9 +618,12 @@ func (b *BlockedEvals) UnblockFailed() {
}
}
if l := len(unblocked); l > 0 {
b.stats.TotalBlocked -= l
if len(unblocked) > 0 {
b.stats.TotalQuotaLimit -= quotaLimit
for eval := range unblocked {
b.stats.Unblock(eval)
}
b.evalBroker.EnqueueAll(unblocked)
}
}
@@ -683,6 +674,7 @@ func (b *BlockedEvals) Flush() {
b.stats.TotalEscaped = 0
b.stats.TotalBlocked = 0
b.stats.TotalQuotaLimit = 0
b.stats.BlockedResources = NewBlockedResourcesStats()
b.captured = make(map[string]wrappedEval)
b.escaped = make(map[string]wrappedEval)
b.jobs = make(map[structs.NamespacedID]string)
@@ -698,7 +690,7 @@ func (b *BlockedEvals) Flush() {
// Stats is used to query the state of the blocked eval tracker.
func (b *BlockedEvals) Stats() *BlockedStats {
// Allocate a new stats struct
stats := new(BlockedStats)
stats := NewBlockedStats()
b.l.RLock()
defer b.l.RUnlock()
@@ -707,6 +699,8 @@ func (b *BlockedEvals) Stats() *BlockedStats {
stats.TotalEscaped = b.stats.TotalEscaped
stats.TotalBlocked = b.stats.TotalBlocked
stats.TotalQuotaLimit = b.stats.TotalQuotaLimit
stats.BlockedResources = b.stats.BlockedResources.Copy()
return stats
}
@@ -719,6 +713,24 @@ func (b *BlockedEvals) EmitStats(period time.Duration, stopCh <-chan struct{}) {
metrics.SetGauge([]string{"nomad", "blocked_evals", "total_quota_limit"}, float32(stats.TotalQuotaLimit))
metrics.SetGauge([]string{"nomad", "blocked_evals", "total_blocked"}, float32(stats.TotalBlocked))
metrics.SetGauge([]string{"nomad", "blocked_evals", "total_escaped"}, float32(stats.TotalEscaped))
for k, v := range stats.BlockedResources.ByJob {
labels := []metrics.Label{
{Name: "namespace", Value: k.Namespace},
{Name: "job", Value: k.ID},
}
metrics.SetGaugeWithLabels([]string{"nomad", "blocked_evals", "job", "cpu"}, float32(v.CPU), labels)
metrics.SetGaugeWithLabels([]string{"nomad", "blocked_evals", "job", "memory"}, float32(v.MemoryMB), labels)
}
for k, v := range stats.BlockedResources.ByNodeInfo {
labels := []metrics.Label{
{Name: "datacenter", Value: k.Datacenter},
{Name: "node_class", Value: k.NodeClass},
}
metrics.SetGaugeWithLabels([]string{"nomad", "blocked_evals", "cpu"}, float32(v.CPU), labels)
metrics.SetGaugeWithLabels([]string{"nomad", "blocked_evals", "memory"}, float32(v.MemoryMB), labels)
}
case <-stopCh:
return
}
@@ -734,15 +746,17 @@ func (b *BlockedEvals) prune(stopCh <-chan struct{}) {
select {
case <-stopCh:
return
case <-ticker.C:
b.pruneUnblockIndexes()
case t := <-ticker.C:
cutoff := t.UTC().Add(-1 * pruneThreshold)
b.pruneUnblockIndexes(cutoff)
b.pruneStats(cutoff)
}
}
}
// pruneUnblockIndexes is used to prune any tracked entry that is excessively
// old. This protects againsts unbounded growth of the map.
func (b *BlockedEvals) pruneUnblockIndexes() {
func (b *BlockedEvals) pruneUnblockIndexes(cutoff time.Time) {
b.l.Lock()
defer b.l.Unlock()
@@ -750,12 +764,18 @@ func (b *BlockedEvals) pruneUnblockIndexes() {
return
}
cutoff := time.Now().UTC().Add(-1 * pruneThreshold)
oldThreshold := b.timetable.NearestIndex(cutoff)
for key, index := range b.unblockIndexes {
if index < oldThreshold {
delete(b.unblockIndexes, key)
}
}
}
// pruneStats is used to prune any zero value stats that are excessively old.
func (b *BlockedEvals) pruneStats(cutoff time.Time) {
b.l.Lock()
defer b.l.Unlock()
b.stats.prune(cutoff)
}

View File

@@ -0,0 +1,209 @@
package nomad
import (
"time"
"github.com/hashicorp/nomad/nomad/structs"
)
// BlockedStats returns all the stats about the blocked eval tracker.
type BlockedStats struct {
// TotalEscaped is the total number of blocked evaluations that have escaped
// computed node classes.
TotalEscaped int
// TotalBlocked is the total number of blocked evaluations.
TotalBlocked int
// TotalQuotaLimit is the total number of blocked evaluations that are due
// to the quota limit being reached.
TotalQuotaLimit int
// BlockedResources stores the amount of resources requested by blocked
// evaluations.
BlockedResources BlockedResourcesStats
}
// NewBlockedStats returns a new BlockedStats.
func NewBlockedStats() *BlockedStats {
return &BlockedStats{
BlockedResources: NewBlockedResourcesStats(),
}
}
// Block updates the stats for the blocked eval tracker with the details of the
// evaluation being blocked.
func (b *BlockedStats) Block(eval *structs.Evaluation) {
b.TotalBlocked++
resourceStats := generateResourceStats(eval)
b.BlockedResources = b.BlockedResources.Add(resourceStats)
}
// Unblock updates the stats for the blocked eval tracker with the details of the
// evaluation being unblocked.
func (b *BlockedStats) Unblock(eval *structs.Evaluation) {
b.TotalBlocked--
resourceStats := generateResourceStats(eval)
b.BlockedResources = b.BlockedResources.Subtract(resourceStats)
}
// prune deletes any key zero metric values older than the cutoff.
func (b *BlockedStats) prune(cutoff time.Time) {
shouldPrune := func(s BlockedResourcesSummary) bool {
return s.Timestamp.Before(cutoff) && s.IsZero()
}
for k, v := range b.BlockedResources.ByJob {
if shouldPrune(v) {
delete(b.BlockedResources.ByJob, k)
}
}
for k, v := range b.BlockedResources.ByNodeInfo {
if shouldPrune(v) {
delete(b.BlockedResources.ByNodeInfo, k)
}
}
}
// generateResourceStats returns a summary of the resources requested by the
// input evaluation.
func generateResourceStats(eval *structs.Evaluation) BlockedResourcesStats {
dcs := make(map[string]struct{})
classes := make(map[string]struct{})
resources := BlockedResourcesSummary{
Timestamp: time.Now().UTC(),
}
for _, allocMetrics := range eval.FailedTGAllocs {
for dc := range allocMetrics.NodesAvailable {
dcs[dc] = struct{}{}
}
for class := range allocMetrics.ClassExhausted {
classes[class] = struct{}{}
}
for _, r := range allocMetrics.ResourcesExhausted {
resources.CPU += r.CPU
resources.MemoryMB += r.MemoryMB
}
}
byJob := make(map[structs.NamespacedID]BlockedResourcesSummary)
byJob[structs.NewNamespacedID(eval.JobID, eval.Namespace)] = resources
byNodeInfo := make(map[NodeInfo]BlockedResourcesSummary)
for dc := range dcs {
for class := range classes {
k := NodeInfo{dc, class}
byNodeInfo[k] = resources
}
}
return BlockedResourcesStats{
ByJob: byJob,
ByNodeInfo: byNodeInfo,
}
}
// BlockedResourcesStats stores resources requested by block evaluations
// split into different dimensions.
type BlockedResourcesStats struct {
ByJob map[structs.NamespacedID]BlockedResourcesSummary
ByNodeInfo map[NodeInfo]BlockedResourcesSummary
}
// NewBlockedResourcesStats returns a new BlockedResourcesStats.
func NewBlockedResourcesStats() BlockedResourcesStats {
return BlockedResourcesStats{
ByJob: make(map[structs.NamespacedID]BlockedResourcesSummary),
ByNodeInfo: make(map[NodeInfo]BlockedResourcesSummary),
}
}
// Copy returns a deep copy of the blocked resource stats.
func (b BlockedResourcesStats) Copy() BlockedResourcesStats {
result := NewBlockedResourcesStats()
for k, v := range b.ByJob {
result.ByJob[k] = v
}
for k, v := range b.ByNodeInfo {
result.ByNodeInfo[k] = v
}
return result
}
// Add returns a new BlockedResourcesStats with the values set to the current
// resource values plus the input.
func (b BlockedResourcesStats) Add(a BlockedResourcesStats) BlockedResourcesStats {
result := b.Copy()
for k, v := range a.ByJob {
result.ByJob[k] = b.ByJob[k].Add(v)
}
for k, v := range a.ByNodeInfo {
result.ByNodeInfo[k] = b.ByNodeInfo[k].Add(v)
}
return result
}
// Subtract returns a new BlockedResourcesStats with the values set to the
// current resource values minus the input.
func (b BlockedResourcesStats) Subtract(a BlockedResourcesStats) BlockedResourcesStats {
result := b.Copy()
for k, v := range a.ByJob {
result.ByJob[k] = b.ByJob[k].Subtract(v)
}
for k, v := range a.ByNodeInfo {
result.ByNodeInfo[k] = b.ByNodeInfo[k].Subtract(v)
}
return result
}
// NodeInfo stores information related to nodes.
type NodeInfo struct {
Datacenter string
NodeClass string
}
// BlockedResourcesSummary stores resource values for blocked evals.
type BlockedResourcesSummary struct {
Timestamp time.Time
CPU int
MemoryMB int
}
// Add returns a new BlockedResourcesSummary with each resource set to the
// current value plus the input.
func (b BlockedResourcesSummary) Add(a BlockedResourcesSummary) BlockedResourcesSummary {
return BlockedResourcesSummary{
Timestamp: a.Timestamp,
CPU: b.CPU + a.CPU,
MemoryMB: b.MemoryMB + a.MemoryMB,
}
}
// Subtract returns a new BlockedResourcesSummary with each resource set to the
// current value minus the input.
func (b BlockedResourcesSummary) Subtract(a BlockedResourcesSummary) BlockedResourcesSummary {
return BlockedResourcesSummary{
Timestamp: a.Timestamp,
CPU: b.CPU - a.CPU,
MemoryMB: b.MemoryMB - a.MemoryMB,
}
}
// IsZero returns true if all resource values are zero.
func (b BlockedResourcesSummary) IsZero() bool {
return b.CPU == 0 && b.MemoryMB == 0
}

View File

@@ -0,0 +1,156 @@
package nomad
import (
"fmt"
"math/rand"
"reflect"
"testing"
"testing/quick"
"time"
"github.com/hashicorp/nomad/nomad/mock"
"github.com/hashicorp/nomad/nomad/structs"
)
// testBlockedEvalsRandomBlockedEval wraps an eval that is randomly generated.
type testBlockedEvalsRandomBlockedEval struct {
eval *structs.Evaluation
}
// Generate returns a random eval.
func (t testBlockedEvalsRandomBlockedEval) Generate(rand *rand.Rand, _ int) reflect.Value {
resourceTypes := []string{"cpu", "memory"}
// Start with a mock eval.
e := mock.BlockedEval()
// Get how many task groups, datacenters and node classes to generate.
// Add 1 to avoid 0.
tgCount := rand.Intn(10) + 1
dcCount := rand.Intn(3) + 1
nodeClassCount := rand.Intn(3) + 1
failedTGAllocs := map[string]*structs.AllocMetric{}
for tg := 1; tg <= tgCount; tg++ {
tgName := fmt.Sprintf("group-%d", tg)
// Get which resource type to use for this task group.
// Nomad stops at the first dimension that is exhausted, so only 1 is
// added per task group.
i := rand.Int() % len(resourceTypes)
resourceType := resourceTypes[i]
failedTGAllocs[tgName] = &structs.AllocMetric{
DimensionExhausted: map[string]int{
resourceType: 1,
},
NodesAvailable: map[string]int{},
ClassExhausted: map[string]int{},
}
for dc := 1; dc <= dcCount; dc++ {
dcName := fmt.Sprintf("dc%d", dc)
failedTGAllocs[tgName].NodesAvailable[dcName] = 1
}
for nc := 1; nc <= nodeClassCount; nc++ {
nodeClassName := fmt.Sprintf("node-class-%d", nc)
failedTGAllocs[tgName].ClassExhausted[nodeClassName] = 1
}
// Generate resources for each task.
taskCount := rand.Intn(5) + 1
resourcesExhausted := map[string]*structs.Resources{}
for t := 1; t <= taskCount; t++ {
task := fmt.Sprintf("task-%d", t)
resourcesExhausted[task] = &structs.Resources{}
resourceAmount := rand.Intn(1000)
switch resourceType {
case "cpu":
resourcesExhausted[task].CPU = resourceAmount
case "memory":
resourcesExhausted[task].MemoryMB = resourceAmount
}
}
failedTGAllocs[tgName].ResourcesExhausted = resourcesExhausted
}
e.FailedTGAllocs = failedTGAllocs
t.eval = e
return reflect.ValueOf(t)
}
// clearTimestampFromBlockedResourceStats set timestamp metrics to zero to
// avoid invalid comparisons.
func clearTimestampFromBlockedResourceStats(b *BlockedResourcesStats) {
for k, v := range b.ByJob {
v.Timestamp = time.Time{}
b.ByJob[k] = v
}
for k, v := range b.ByNodeInfo {
v.Timestamp = time.Time{}
b.ByNodeInfo[k] = v
}
}
// TestBlockedEvalsStats_BlockedResources generates random evals and processes
// them using the expected code paths and a manual check of the expeceted result.
func TestBlockedEvalsStats_BlockedResources(t *testing.T) {
t.Parallel()
blocked, _ := testBlockedEvals(t)
// evalHistory stores all evals generated during the test.
evalHistory := []*structs.Evaluation{}
// blockedEvals keeps track if evals are blocked or unblocked.
blockedEvals := map[string]bool{}
// blockAndUntrack processes the generated evals in order using a
// BlockedEvals instance.
blockAndUntrack := func(testEval testBlockedEvalsRandomBlockedEval, block bool, unblockIdx uint16) BlockedResourcesStats {
if block || len(evalHistory) == 0 {
blocked.Block(testEval.eval)
} else {
i := int(unblockIdx) % len(evalHistory)
eval := evalHistory[i]
blocked.Untrack(eval.JobID, eval.Namespace)
}
// Remove zero stats from unblocked evals.
blocked.pruneStats(time.Now().UTC())
result := blocked.Stats().BlockedResources
clearTimestampFromBlockedResourceStats(&result)
return result
}
// manualCount processes only the blocked evals and generate a
// BlockedResourcesStats result directly from the eval history.
manualCount := func(testEval testBlockedEvalsRandomBlockedEval, block bool, unblockIdx uint16) BlockedResourcesStats {
if block || len(evalHistory) == 0 {
evalHistory = append(evalHistory, testEval.eval)
blockedEvals[testEval.eval.ID] = true
} else {
i := int(unblockIdx) % len(evalHistory)
eval := evalHistory[i]
blockedEvals[eval.ID] = false
}
result := NewBlockedResourcesStats()
for _, e := range evalHistory {
if !blockedEvals[e.ID] {
continue
}
result = result.Add(generateResourceStats(e))
}
clearTimestampFromBlockedResourceStats(&result)
return result
}
err := quick.CheckEqual(blockAndUntrack, manualCount, nil)
if err != nil {
t.Error(err)
}
}

View File

@@ -2,7 +2,6 @@ package nomad
import (
"fmt"
"reflect"
"testing"
"time"
@@ -23,108 +22,115 @@ func testBlockedEvals(t *testing.T) (*BlockedEvals, *EvalBroker) {
func TestBlockedEvals_Block_Disabled(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, _ := testBlockedEvals(t)
blocked.SetEnabled(false)
// Create an escaped eval and add it to the blocked tracker.
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
e := mock.BlockedEval()
e.EscapedComputedClass = true
blocked.Block(e)
// Verify block did nothing
bStats := blocked.Stats()
if bStats.TotalBlocked != 0 || bStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", bStats)
}
// Verify block did nothing.
blockedStats := blocked.Stats()
require.Equal(0, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 0)
}
func TestBlockedEvals_Block_SameJob(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, _ := testBlockedEvals(t)
// Create two blocked evals and add them to the blocked tracker.
e := mock.Eval()
e2 := mock.Eval()
e := mock.BlockedEval()
e2 := mock.BlockedEval()
e2.JobID = e.JobID
blocked.Block(e)
blocked.Block(e2)
// Verify block did track both
bStats := blocked.Stats()
if bStats.TotalBlocked != 1 || bStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", bStats)
}
// Verify block didn't track duplicate.
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
}
func TestBlockedEvals_Block_Quota(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, _ := testBlockedEvals(t)
// Create a blocked evals on quota
e := mock.Eval()
// Create a blocked eval on quota.
e := mock.BlockedEval()
e.QuotaLimitReached = "foo"
blocked.Block(e)
// Verify block did track both
bs := blocked.Stats()
if bs.TotalBlocked != 1 || bs.TotalEscaped != 0 || bs.TotalQuotaLimit != 1 {
t.Fatalf("bad: %#v", bs)
}
// Verify block did track eval.
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Equal(1, blockedStats.TotalQuotaLimit)
}
func TestBlockedEvals_Block_PriorUnblocks(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, _ := testBlockedEvals(t)
// Do unblocks prior to blocking
// Do unblocks prior to blocking.
blocked.Unblock("v1:123", 1000)
blocked.Unblock("v1:123", 1001)
// Create two blocked evals and add them to the blocked tracker.
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
// Create blocked eval with two classes ineligible.
e := mock.BlockedEval()
e.ClassEligibility = map[string]bool{"v1:123": false, "v1:456": false}
e.SnapshotIndex = 999
blocked.Block(e)
// Verify block did track both
bStats := blocked.Stats()
if bStats.TotalBlocked != 1 || bStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", bStats)
}
// Verify block did track eval.
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
}
func TestBlockedEvals_GetDuplicates(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, _ := testBlockedEvals(t)
// Create duplicate blocked evals and add them to the blocked tracker.
e := mock.Eval()
e := mock.BlockedEval()
e.CreateIndex = 100
e2 := mock.Eval()
e2 := mock.BlockedEval()
e2.JobID = e.JobID
e2.CreateIndex = 101
e3 := mock.Eval()
e3 := mock.BlockedEval()
e3.JobID = e.JobID
e3.CreateIndex = 102
e4 := mock.Eval()
e4 := mock.BlockedEval()
e4.JobID = e.JobID
e4.CreateIndex = 100
blocked.Block(e)
blocked.Block(e2)
// Verify stats such that we are only tracking one
bStats := blocked.Stats()
if bStats.TotalBlocked != 1 || bStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", bStats)
}
// Verify stats such that we are only tracking one.
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
// Get the duplicates.
out := blocked.GetDuplicates(0)
if len(out) != 1 || !reflect.DeepEqual(out[0], e) {
t.Fatalf("bad: %#v %#v", out, e)
}
require.Len(out, 1)
require.Equal(e, out[0])
// Call block again after a small sleep.
go func() {
@@ -134,45 +140,45 @@ func TestBlockedEvals_GetDuplicates(t *testing.T) {
// Get the duplicates.
out = blocked.GetDuplicates(1 * time.Second)
if len(out) != 1 || !reflect.DeepEqual(out[0], e2) {
t.Fatalf("bad: %#v %#v", out, e2)
}
require.Len(out, 1)
require.Equal(e2, out[0])
// Verify stats such that we are only tracking one
bStats = blocked.Stats()
if bStats.TotalBlocked != 1 || bStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", bStats)
}
// Verify stats such that we are only tracking one.
blockedStats = blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
// Add an older evaluation and assert it gets cancelled
// Add an older evaluation and assert it gets cancelled.
blocked.Block(e4)
out = blocked.GetDuplicates(0)
if len(out) != 1 || !reflect.DeepEqual(out[0], e4) {
t.Fatalf("bad: %#v %#v", out, e4)
}
require.Len(out, 1)
require.Equal(e4, out[0])
// Verify stats such that we are only tracking one
bStats = blocked.Stats()
if bStats.TotalBlocked != 1 || bStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", bStats)
}
// Verify stats such that we are only tracking one.
blockedStats = blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
}
func TestBlockedEvals_UnblockEscaped(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Create an escaped eval and add it to the blocked tracker.
e := mock.Eval()
e := mock.BlockedEval()
e.Status = structs.EvalStatusBlocked
e.EscapedComputedClass = true
blocked.Block(e)
// Verify block caused the eval to be tracked
bStats := blocked.Stats()
if bStats.TotalBlocked != 1 || bStats.TotalEscaped != 1 {
t.Fatalf("bad: %#v", bStats)
}
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(1, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
blocked.Unblock("v1:123", 1000)
requireBlockedEvalsEnqueued(t, blocked, broker, 1)
@@ -186,10 +192,16 @@ func requireBlockedEvalsEnqueued(t *testing.T, blocked *BlockedEvals, broker *Ev
return false, fmt.Errorf("missing enqueued evals: %#v", brokerStats)
}
// Prune old and empty metrics.
blocked.pruneStats(time.Now().UTC())
// Verify Unblock updates the stats
bStats := blocked.Stats()
if bStats.TotalBlocked != 0 || bStats.TotalEscaped != 0 {
return false, fmt.Errorf("evals still blocked: %#v", bStats)
blockedStats := blocked.Stats()
ok := blockedStats.TotalBlocked == 0 &&
blockedStats.TotalEscaped == 0 &&
len(blockedStats.BlockedResources.ByJob) == 0
if !ok {
return false, fmt.Errorf("evals still blocked: %#v", blockedStats)
}
return true, nil
}, func(err error) {
@@ -199,20 +211,20 @@ func requireBlockedEvalsEnqueued(t *testing.T, blocked *BlockedEvals, broker *Ev
func TestBlockedEvals_UnblockEligible(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Create a blocked eval that is eligible on a specific node class and add
// it to the blocked tracker.
e := mock.Eval()
e := mock.BlockedEval()
e.Status = structs.EvalStatusBlocked
e.ClassEligibility = map[string]bool{"v1:123": true}
blocked.Block(e)
// Verify block caused the eval to be tracked
blockedStats := blocked.Stats()
if blockedStats.TotalBlocked != 1 {
t.Fatalf("bad: %#v", blockedStats)
}
require.Equal(1, blockedStats.TotalBlocked)
blocked.Unblock("v1:123", 1000)
requireBlockedEvalsEnqueued(t, blocked, broker, 1)
@@ -220,20 +232,21 @@ func TestBlockedEvals_UnblockEligible(t *testing.T) {
func TestBlockedEvals_UnblockIneligible(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Create a blocked eval that is ineligible on a specific node class and add
// it to the blocked tracker.
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
e := mock.BlockedEval()
e.ClassEligibility = map[string]bool{"v1:123": false}
blocked.Block(e)
// Verify block caused the eval to be tracked
blockedStats := blocked.Stats()
if blockedStats.TotalBlocked != 1 && blockedStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", blockedStats)
}
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
// Should do nothing
blocked.Unblock("v1:123", 1000)
@@ -242,12 +255,18 @@ func TestBlockedEvals_UnblockIneligible(t *testing.T) {
// Verify Unblock didn't cause an enqueue
brokerStats := broker.Stats()
if brokerStats.TotalReady != 0 {
return false, fmt.Errorf("bad: %#v", brokerStats)
return false, fmt.Errorf("eval unblocked: %#v", brokerStats)
}
bStats := blocked.Stats()
if bStats.TotalBlocked != 1 || bStats.TotalEscaped != 0 {
return false, fmt.Errorf("bad: %#v", bStats)
// Prune old and empty metrics.
blocked.pruneStats(time.Now().UTC())
blockedStats := blocked.Stats()
ok := blockedStats.TotalBlocked == 1 &&
blockedStats.TotalEscaped == 0 &&
len(blockedStats.BlockedResources.ByJob) == 1
if !ok {
return false, fmt.Errorf("eval unblocked: %#v", blockedStats)
}
return true, nil
}, func(err error) {
@@ -257,20 +276,21 @@ func TestBlockedEvals_UnblockIneligible(t *testing.T) {
func TestBlockedEvals_UnblockUnknown(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Create a blocked eval that is ineligible on a specific node class and add
// it to the blocked tracker.
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
e := mock.BlockedEval()
e.ClassEligibility = map[string]bool{"v1:123": true, "v1:456": false}
blocked.Block(e)
// Verify block caused the eval to be tracked
// Verify block caused the eval to be tracked.
blockedStats := blocked.Stats()
if blockedStats.TotalBlocked != 1 && blockedStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", blockedStats)
}
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
// Should unblock because the eval hasn't seen this node class.
blocked.Unblock("v1:789", 1000)
@@ -279,19 +299,20 @@ func TestBlockedEvals_UnblockUnknown(t *testing.T) {
func TestBlockedEvals_UnblockEligible_Quota(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Create a blocked eval that is eligible for a particular quota
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
// Create a blocked eval that is eligible for a particular quota.
e := mock.BlockedEval()
e.QuotaLimitReached = "foo"
blocked.Block(e)
// Verify block caused the eval to be tracked
bs := blocked.Stats()
if bs.TotalBlocked != 1 || bs.TotalQuotaLimit != 1 {
t.Fatalf("bad: %#v", bs)
}
// Verify block caused the eval to be tracked.
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(1, blockedStats.TotalQuotaLimit)
require.Len(blockedStats.BlockedResources.ByJob, 1)
blocked.UnblockQuota("foo", 1000)
requireBlockedEvalsEnqueued(t, blocked, broker, 1)
@@ -299,33 +320,41 @@ func TestBlockedEvals_UnblockEligible_Quota(t *testing.T) {
func TestBlockedEvals_UnblockIneligible_Quota(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Create a blocked eval that is eligible on a specific quota
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
// Create a blocked eval that is eligible on a specific quota.
e := mock.BlockedEval()
e.QuotaLimitReached = "foo"
blocked.Block(e)
// Verify block caused the eval to be tracked
bs := blocked.Stats()
if bs.TotalBlocked != 1 || bs.TotalQuotaLimit != 1 {
t.Fatalf("bad: %#v", bs)
}
// Verify block caused the eval to be tracked.
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(1, blockedStats.TotalQuotaLimit)
require.Len(blockedStats.BlockedResources.ByJob, 1)
// Should do nothing
// Should do nothing.
blocked.UnblockQuota("bar", 1000)
testutil.WaitForResult(func() (bool, error) {
// Verify Unblock didn't cause an enqueue
brokerStats := broker.Stats()
if brokerStats.TotalReady != 0 {
return false, fmt.Errorf("bad: %#v", brokerStats)
return false, fmt.Errorf("eval unblocked: %#v", brokerStats)
}
bs := blocked.Stats()
if bs.TotalBlocked != 1 || bs.TotalEscaped != 0 || bs.TotalQuotaLimit != 1 {
return false, fmt.Errorf("bad: %#v", bs)
// Prune old and empty metrics.
blocked.pruneStats(time.Now().UTC())
blockedStats := blocked.Stats()
ok := blockedStats.TotalBlocked == 1 &&
blockedStats.TotalEscaped == 0 &&
blockedStats.TotalQuotaLimit == 1 &&
len(blockedStats.BlockedResources.ByJob) == 1
if !ok {
return false, fmt.Errorf("eval unblocked: %#v", blockedStats)
}
return true, nil
}, func(err error) {
@@ -335,42 +364,39 @@ func TestBlockedEvals_UnblockIneligible_Quota(t *testing.T) {
func TestBlockedEvals_Reblock(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Create an evaluation, Enqueue/Dequeue it to get a token
e := mock.Eval()
e := mock.BlockedEval()
e.SnapshotIndex = 500
e.Status = structs.EvalStatusBlocked
e.ClassEligibility = map[string]bool{"v1:123": true, "v1:456": false}
broker.Enqueue(e)
_, token, err := broker.Dequeue([]string{e.Type}, time.Second)
if err != nil {
t.Fatalf("err: %v", err)
}
require.NoError(err)
// Reblock the evaluation
blocked.Reblock(e, token)
// Verify block caused the eval to be tracked
blockedStats := blocked.Stats()
if blockedStats.TotalBlocked != 1 && blockedStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", blockedStats)
}
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
// Should unblock because the eval
blocked.Unblock("v1:123", 1000)
brokerStats := broker.Stats()
if brokerStats.TotalReady != 0 && brokerStats.TotalUnacked != 1 {
t.Fatalf("bad: %#v", brokerStats)
}
require.Equal(0, brokerStats.TotalReady)
require.Equal(1, brokerStats.TotalUnacked)
// Ack the evaluation which should cause the reblocked eval to transition
// to ready
if err := broker.Ack(e.ID, token); err != nil {
t.Fatalf("err: %v", err)
}
err = broker.Ack(e.ID, token)
require.NoError(err)
requireBlockedEvalsEnqueued(t, blocked, broker, 1)
}
@@ -379,6 +405,8 @@ func TestBlockedEvals_Reblock(t *testing.T) {
// it is escaped and old
func TestBlockedEvals_Block_ImmediateUnblock_Escaped(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Do an unblock prior to blocking
@@ -386,17 +414,16 @@ func TestBlockedEvals_Block_ImmediateUnblock_Escaped(t *testing.T) {
// Create a blocked eval that is eligible on a specific node class and add
// it to the blocked tracker.
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
e := mock.BlockedEval()
e.EscapedComputedClass = true
e.SnapshotIndex = 900
blocked.Block(e)
// Verify block caused the eval to be immediately unblocked
blockedStats := blocked.Stats()
if blockedStats.TotalBlocked != 0 && blockedStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", blockedStats)
}
require.Equal(0, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 0)
requireBlockedEvalsEnqueued(t, blocked, broker, 1)
}
@@ -406,6 +433,8 @@ func TestBlockedEvals_Block_ImmediateUnblock_Escaped(t *testing.T) {
// scheduler
func TestBlockedEvals_Block_ImmediateUnblock_UnseenClass_After(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Do an unblock prior to blocking
@@ -413,17 +442,16 @@ func TestBlockedEvals_Block_ImmediateUnblock_UnseenClass_After(t *testing.T) {
// Create a blocked eval that is eligible on a specific node class and add
// it to the blocked tracker.
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
e := mock.BlockedEval()
e.EscapedComputedClass = false
e.SnapshotIndex = 900
blocked.Block(e)
// Verify block caused the eval to be immediately unblocked
blockedStats := blocked.Stats()
if blockedStats.TotalBlocked != 0 && blockedStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", blockedStats)
}
require.Equal(0, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 0)
requireBlockedEvalsEnqueued(t, blocked, broker, 1)
}
@@ -433,6 +461,8 @@ func TestBlockedEvals_Block_ImmediateUnblock_UnseenClass_After(t *testing.T) {
// scheduler
func TestBlockedEvals_Block_ImmediateUnblock_UnseenClass_Before(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, _ := testBlockedEvals(t)
// Do an unblock prior to blocking
@@ -440,23 +470,24 @@ func TestBlockedEvals_Block_ImmediateUnblock_UnseenClass_Before(t *testing.T) {
// Create a blocked eval that is eligible on a specific node class and add
// it to the blocked tracker.
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
e := mock.BlockedEval()
e.EscapedComputedClass = false
e.SnapshotIndex = 900
blocked.Block(e)
// Verify block caused the eval to be immediately unblocked
blockedStats := blocked.Stats()
if blockedStats.TotalBlocked != 1 && blockedStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", blockedStats)
}
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
}
// Test the block case in which the eval should be immediately unblocked since
// it a class it is eligible for has been unblocked
func TestBlockedEvals_Block_ImmediateUnblock_SeenClass(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Do an unblock prior to blocking
@@ -464,17 +495,16 @@ func TestBlockedEvals_Block_ImmediateUnblock_SeenClass(t *testing.T) {
// Create a blocked eval that is eligible on a specific node class and add
// it to the blocked tracker.
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
e := mock.BlockedEval()
e.ClassEligibility = map[string]bool{"v1:123": true, "v1:456": false}
e.SnapshotIndex = 900
blocked.Block(e)
// Verify block caused the eval to be immediately unblocked
blockedStats := blocked.Stats()
if blockedStats.TotalBlocked != 0 && blockedStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", blockedStats)
}
require.Equal(0, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 0)
requireBlockedEvalsEnqueued(t, blocked, broker, 1)
}
@@ -483,6 +513,8 @@ func TestBlockedEvals_Block_ImmediateUnblock_SeenClass(t *testing.T) {
// it a quota has changed that it is using
func TestBlockedEvals_Block_ImmediateUnblock_Quota(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Do an unblock prior to blocking
@@ -490,40 +522,40 @@ func TestBlockedEvals_Block_ImmediateUnblock_Quota(t *testing.T) {
// Create a blocked eval that is eligible on a specific node class and add
// it to the blocked tracker.
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
e := mock.BlockedEval()
e.QuotaLimitReached = "my-quota"
e.SnapshotIndex = 900
blocked.Block(e)
// Verify block caused the eval to be immediately unblocked
bs := blocked.Stats()
if bs.TotalBlocked != 0 && bs.TotalEscaped != 0 && bs.TotalQuotaLimit != 0 {
t.Fatalf("bad: %#v", bs)
}
blockedStats := blocked.Stats()
require.Equal(0, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Equal(0, blockedStats.TotalQuotaLimit)
require.Len(blockedStats.BlockedResources.ByJob, 0)
requireBlockedEvalsEnqueued(t, blocked, broker, 1)
}
func TestBlockedEvals_UnblockFailed(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
// Create blocked evals that are due to failures
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
e := mock.BlockedEval()
e.TriggeredBy = structs.EvalTriggerMaxPlans
e.EscapedComputedClass = true
blocked.Block(e)
e2 := mock.Eval()
e2 := mock.BlockedEval()
e2.Status = structs.EvalStatusBlocked
e2.TriggeredBy = structs.EvalTriggerMaxPlans
e2.ClassEligibility = map[string]bool{"v1:123": true, "v1:456": false}
blocked.Block(e2)
e3 := mock.Eval()
e3.Status = structs.EvalStatusBlocked
e3 := mock.BlockedEval()
e3.TriggeredBy = structs.EvalTriggerMaxPlans
e3.QuotaLimitReached = "foo"
blocked.Block(e3)
@@ -531,98 +563,116 @@ func TestBlockedEvals_UnblockFailed(t *testing.T) {
// Trigger an unblock fail
blocked.UnblockFailed()
// Prune old and empty metrics.
blocked.pruneStats(time.Now().UTC())
// Verify UnblockFailed caused the eval to be immediately unblocked
bs := blocked.Stats()
if bs.TotalBlocked != 0 || bs.TotalEscaped != 0 || bs.TotalQuotaLimit != 0 {
t.Fatalf("bad: %#v", bs)
}
blockedStats := blocked.Stats()
require.Equal(0, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Equal(0, blockedStats.TotalQuotaLimit)
require.Len(blockedStats.BlockedResources.ByJob, 0)
requireBlockedEvalsEnqueued(t, blocked, broker, 3)
// Reblock an eval for the same job and check that it gets tracked.
blocked.Block(e)
bs = blocked.Stats()
if bs.TotalBlocked != 1 || bs.TotalEscaped != 1 {
t.Fatalf("bad: %#v", bs)
}
blockedStats = blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(1, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
}
func TestBlockedEvals_Untrack(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, _ := testBlockedEvals(t)
// Create two blocked evals and add them to the blocked tracker.
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
// Create blocked eval and add to the blocked tracker.
e := mock.BlockedEval()
e.ClassEligibility = map[string]bool{"v1:123": false, "v1:456": false}
e.SnapshotIndex = 1000
blocked.Block(e)
// Verify block did track
bStats := blocked.Stats()
if bStats.TotalBlocked != 1 || bStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", bStats)
}
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
// Untrack and verify
blocked.Untrack(e.JobID, e.Namespace)
bStats = blocked.Stats()
if bStats.TotalBlocked != 0 || bStats.TotalEscaped != 0 {
t.Fatalf("bad: %#v", bStats)
}
blocked.pruneStats(time.Now().UTC())
blockedStats = blocked.Stats()
require.Equal(0, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 0)
}
func TestBlockedEvals_Untrack_Quota(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, _ := testBlockedEvals(t)
// Create a blocked evals and add it to the blocked tracker.
e := mock.Eval()
e.Status = structs.EvalStatusBlocked
// Create a blocked eval and add it to the blocked tracker.
e := mock.BlockedEval()
e.QuotaLimitReached = "foo"
e.SnapshotIndex = 1000
blocked.Block(e)
// Verify block did track
bs := blocked.Stats()
if bs.TotalBlocked != 1 || bs.TotalEscaped != 0 || bs.TotalQuotaLimit != 1 {
t.Fatalf("bad: %#v", bs)
}
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 1)
// Untrack and verify
blocked.Untrack(e.JobID, e.Namespace)
bs = blocked.Stats()
if bs.TotalBlocked != 0 || bs.TotalEscaped != 0 || bs.TotalQuotaLimit != 0 {
t.Fatalf("bad: %#v", bs)
}
blocked.pruneStats(time.Now().UTC())
blockedStats = blocked.Stats()
require.Equal(0, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Len(blockedStats.BlockedResources.ByJob, 0)
}
func TestBlockedEvals_UnblockNode(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, broker := testBlockedEvals(t)
require.NotNil(t, broker)
// Create a blocked evals and add it to the blocked tracker.
e := mock.Eval()
e := mock.BlockedEval()
e.Type = structs.JobTypeSystem
e.NodeID = "foo"
e.SnapshotIndex = 999
blocked.Block(e)
// Verify block did track
bs := blocked.Stats()
require.Equal(t, 1, bs.TotalBlocked)
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Len(blockedStats.BlockedResources.ByJob, 1)
blocked.UnblockNode("foo", 1000)
requireBlockedEvalsEnqueued(t, blocked, broker, 1)
bs = blocked.Stats()
require.Empty(t, blocked.system.byNode)
require.Equal(t, 0, bs.TotalBlocked)
blocked.pruneStats(time.Now().UTC())
blockedStats = blocked.Stats()
require.Empty(blocked.system.byNode)
require.Equal(0, blockedStats.TotalBlocked)
require.Len(blockedStats.BlockedResources.ByJob, 0)
}
func TestBlockedEvals_SystemUntrack(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, _ := testBlockedEvals(t)
// Create a blocked evals and add it to the blocked tracker.
@@ -632,21 +682,26 @@ func TestBlockedEvals_SystemUntrack(t *testing.T) {
blocked.Block(e)
// Verify block did track
bs := blocked.Stats()
require.Equal(t, 1, bs.TotalBlocked)
require.Equal(t, 0, bs.TotalEscaped)
require.Equal(t, 0, bs.TotalQuotaLimit)
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Equal(0, blockedStats.TotalQuotaLimit)
require.Len(blockedStats.BlockedResources.ByJob, 1)
// Untrack and verify
blocked.Untrack(e.JobID, e.Namespace)
bs = blocked.Stats()
require.Equal(t, 0, bs.TotalBlocked)
require.Equal(t, 0, bs.TotalEscaped)
require.Equal(t, 0, bs.TotalQuotaLimit)
blocked.pruneStats(time.Now().UTC())
blockedStats = blocked.Stats()
require.Equal(0, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Equal(0, blockedStats.TotalQuotaLimit)
require.Len(blockedStats.BlockedResources.ByJob, 0)
}
func TestBlockedEvals_SystemDisableFlush(t *testing.T) {
t.Parallel()
require := require.New(t)
blocked, _ := testBlockedEvals(t)
// Create a blocked evals and add it to the blocked tracker.
@@ -656,18 +711,20 @@ func TestBlockedEvals_SystemDisableFlush(t *testing.T) {
blocked.Block(e)
// Verify block did track
bs := blocked.Stats()
require.Equal(t, 1, bs.TotalBlocked)
require.Equal(t, 0, bs.TotalEscaped)
require.Equal(t, 0, bs.TotalQuotaLimit)
blockedStats := blocked.Stats()
require.Equal(1, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Equal(0, blockedStats.TotalQuotaLimit)
require.Len(blockedStats.BlockedResources.ByJob, 1)
// Disable empties
blocked.SetEnabled(false)
bs = blocked.Stats()
require.Equal(t, 0, bs.TotalBlocked)
require.Equal(t, 0, bs.TotalEscaped)
require.Equal(t, 0, bs.TotalQuotaLimit)
require.Empty(t, blocked.system.evals)
require.Empty(t, blocked.system.byJob)
require.Empty(t, blocked.system.byNode)
blockedStats = blocked.Stats()
require.Equal(0, blockedStats.TotalBlocked)
require.Equal(0, blockedStats.TotalEscaped)
require.Equal(0, blockedStats.TotalQuotaLimit)
require.Len(blockedStats.BlockedResources.ByJob, 0)
require.Empty(blocked.system.evals)
require.Empty(blocked.system.byJob)
require.Empty(blocked.system.byNode)
}

View File

@@ -1092,6 +1092,26 @@ func Eval() *structs.Evaluation {
return eval
}
func BlockedEval() *structs.Evaluation {
e := Eval()
e.Status = structs.EvalStatusBlocked
e.FailedTGAllocs = map[string]*structs.AllocMetric{
"cache": {
DimensionExhausted: map[string]int{
"memory": 1,
},
ResourcesExhausted: map[string]*structs.Resources{
"redis": {
CPU: 100,
MemoryMB: 1024,
},
},
},
}
return e
}
func JobSummary(jobID string) *structs.JobSummary {
js := &structs.JobSummary{
JobID: jobID,

View File

@@ -9749,6 +9749,10 @@ type AllocMetric struct {
// QuotaExhausted provides the exhausted dimensions
QuotaExhausted []string
// ResourcesExhausted provides the amount of resources exhausted by task
// during the allocation placement
ResourcesExhausted map[string]*Resources
// Scores is the scores of the final few nodes remaining
// for placement. The top score is typically selected.
// Deprecated: Replaced by ScoreMetaData in Nomad 0.9
@@ -9837,6 +9841,35 @@ func (a *AllocMetric) ExhaustQuota(dimensions []string) {
a.QuotaExhausted = append(a.QuotaExhausted, dimensions...)
}
// ExhaustResources updates the amount of resources exhausted for the
// allocation because of the given task group.
func (a *AllocMetric) ExhaustResources(tg *TaskGroup) {
if a.DimensionExhausted == nil {
return
}
if a.ResourcesExhausted == nil {
a.ResourcesExhausted = make(map[string]*Resources)
}
for _, t := range tg.Tasks {
exhaustedResources := a.ResourcesExhausted[t.Name]
if exhaustedResources == nil {
exhaustedResources = &Resources{}
}
if a.DimensionExhausted["memory"] > 0 {
exhaustedResources.MemoryMB += t.Resources.MemoryMB
}
if a.DimensionExhausted["cpu"] > 0 {
exhaustedResources.CPU += t.Resources.CPU
}
a.ResourcesExhausted[t.Name] = exhaustedResources
}
}
// ScoreNode is used to gather top K scoring nodes in a heap
func (a *AllocMetric) ScoreNode(node *Node, name string, score float64) {
// Create nodeScoreMeta lazily if its the first time or if its a new node
@@ -10308,7 +10341,7 @@ func (e *Evaluation) NextRollingEval(wait time.Duration) *Evaluation {
// ineligible, whether the job has escaped computed node classes and whether the
// quota limit was reached.
func (e *Evaluation) CreateBlockedEval(classEligibility map[string]bool,
escaped bool, quotaReached string) *Evaluation {
escaped bool, quotaReached string, failedTGAllocs map[string]*AllocMetric) *Evaluation {
now := time.Now().UTC().UnixNano()
return &Evaluation{
ID: uuid.Generate(),
@@ -10320,6 +10353,7 @@ func (e *Evaluation) CreateBlockedEval(classEligibility map[string]bool,
JobModifyIndex: e.JobModifyIndex,
Status: EvalStatusBlocked,
PreviousEval: e.ID,
FailedTGAllocs: failedTGAllocs,
ClassEligibility: classEligibility,
EscapedComputedClass: escaped,
QuotaLimitReached: quotaReached,

View File

@@ -200,7 +200,7 @@ func (s *GenericScheduler) createBlockedEval(planFailure bool) error {
classEligibility = e.GetClasses()
}
s.blocked = s.eval.CreateBlockedEval(classEligibility, escaped, e.QuotaLimitReached())
s.blocked = s.eval.CreateBlockedEval(classEligibility, escaped, e.QuotaLimitReached(), s.failedTGAllocs)
if planFailure {
s.blocked.TriggeredBy = structs.EvalTriggerMaxPlans
s.blocked.StatusDescription = blockedEvalMaxPlanDesc
@@ -520,6 +520,7 @@ func (s *GenericScheduler) computePlacements(destructive, place []placementResul
// Check if this task group has already failed
if metric, ok := s.failedTGAllocs[tg.Name]; ok {
metric.CoalescedFailures += 1
metric.ExhaustResources(tg)
continue
}
@@ -627,6 +628,9 @@ func (s *GenericScheduler) computePlacements(destructive, place []placementResul
s.failedTGAllocs = make(map[string]*structs.AllocMetric)
}
// Update metrics with the resources requested by the task group.
s.ctx.Metrics().ExhaustResources(tg)
// Track the fact that we didn't find a placement
s.failedTGAllocs[tg.Name] = s.ctx.Metrics()

View File

@@ -311,6 +311,7 @@ func (s *SystemScheduler) computePlacements(place []allocTuple) error {
// Check if this task group has already failed, reported to the user as a count
if metric, ok := s.failedTGAllocs[missing.TaskGroup.Name]; ok {
metric.CoalescedFailures += 1
metric.ExhaustResources(missing.TaskGroup)
continue
}
@@ -325,6 +326,9 @@ func (s *SystemScheduler) computePlacements(place []allocTuple) error {
s.failedTGAllocs = make(map[string]*structs.AllocMetric)
}
// Update metrics with the resources requested by the task group.
s.ctx.Metrics().ExhaustResources(missing.TaskGroup)
// Actual failure to start this task on this candidate node, report it individually
s.failedTGAllocs[missing.TaskGroup.Name] = s.ctx.Metrics()
s.addBlocked(node)
@@ -417,7 +421,7 @@ func (s *SystemScheduler) addBlocked(node *structs.Node) error {
classEligibility = e.GetClasses()
}
blocked := s.eval.CreateBlockedEval(classEligibility, escaped, e.QuotaLimitReached())
blocked := s.eval.CreateBlockedEval(classEligibility, escaped, e.QuotaLimitReached(), s.failedTGAllocs)
blocked.StatusDescription = blockedEvalFailedPlacements
blocked.NodeID = node.ID

View File

@@ -374,6 +374,7 @@ type AllocationMetric struct {
ClassExhausted map[string]int
DimensionExhausted map[string]int
QuotaExhausted []string
ResourcesExhausted map[string]*Resources
// Deprecated, replaced with ScoreMetaData
Scores map[string]float64
AllocationTime time.Duration

View File

@@ -228,214 +228,218 @@ Job status metrics are emitted by the Nomad leader server.
The following table includes metrics for overall cluster health in addition to
those listed in [Key Metrics](#key-metrics) above.
| Metric | Description | Unit | Type | Labels |
| ---------------------------------------------------- | ----------------------------------------------------------------- | -------------------- | ------- | ------ |
| `nomad.memberlist.gossip` | Time elapsed to broadcast gossip messages | Nanoseconds | Summary | host |
| `nomad.nomad.acl.bootstrap` | Time elapsed for `ACL.Bootstrap` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.delete_policies` | Time elapsed for `ACL.DeletePolicies` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.delete_tokens` | Time elapsed for `ACL.DeleteTokens` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.get_policies` | Time elapsed for `ACL.GetPolicies` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.get_policy` | Time elapsed for `ACL.GetPolicy` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.get_token` | Time elapsed for `ACL.GetToken` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.get_tokens` | Time elapsed for `ACL.GetTokens` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.list_policies` | Time elapsed for `ACL.ListPolicies` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.list_tokens` | Time elapsed for `ACL.ListTokens` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.resolve_token` | Time elapsed for `ACL.ResolveToken` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.upsert_policies` | Time elapsed for `ACL.UpsertPolicies` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.upsert_tokens` | Time elapsed for `ACL.UpsertTokens` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.exec` | Time elapsed to establish alloc exec | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.get_alloc` | Time elapsed for `Alloc.GetAlloc` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.get_allocs` | Time elapsed for `Alloc.GetAllocs` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.list` | Time elapsed for `Alloc.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.stop` | Time elapsed for `Alloc.Stop` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.update_desired_transition` | Time elapsed for `Alloc.UpdateDesiredTransition` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.blocked_evals.total_blocked` | Count of evals in the blocked state | Integer | Gauge | host |
| `nomad.nomad.blocked_evals.total_escaped` | Count of evals that have escaped computed node classes | Integer | Gauge | host |
| `nomad.nomad.blocked_evals.total_quota_limit` | Count of blocked evals due to quota limits | Integer | Gauge | host |
| `nomad.nomad.broker.batch_ready` | Count of batch evals ready to be scheduled | Integer | Gauge | host |
| `nomad.nomad.broker.batch_unacked` | Count of unacknowledged batch evals | Integer | Gauge | host |
| `nomad.nomad.broker.service_ready` | Count of service evals ready to be scheduled | Integer | Gauge | host |
| `nomad.nomad.broker.service_unacked` | Count of unacknowledged service evals | Integer | Gauge | host |
| `nomad.nomad.broker.system_ready` | Count of system evals ready to be scheduled | Integer | Gauge | host |
| `nomad.nomad.broker.system_unacked` | Count of unacknowledged system evals | Integer | Gauge | host |
| `nomad.nomad.broker.total_ready` | Count of evals in the ready state | Integer | Gauge | host |
| `nomad.nomad.broker.total_waiting` | Count of evals in the waiting state | Integer | Gauge | host |
| `nomad.nomad.client.batch_deregister` | Time elapsed for `Node.BatchDeregister` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.deregister` | Time elapsed for `Node.Deregister` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.derive_si_token` | Time elapsed for `Node.DeriveSIToken` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.derive_vault_token` | Time elapsed for `Node.DeriveVaultToken` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.emit_events` | Time elapsed for `Node.EmitEvents` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.evaluate` | Time elapsed for `Node.Evaluate` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.get_allocs` | Time elapsed for `Node.GetAllocs` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.get_client_allocs` | Time elapsed for `Node.GetClientAllocs` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.get_node` | Time elapsed for `Node.GetNode` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.list` | Time elapsed for `Node.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.register` | Time elapsed for `Node.Register` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.stats` | Time elapsed for `Client.Stats` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.update_alloc` | Time elapsed for `Node.UpdateAlloc` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.update_drain` | Time elapsed for `Node.UpdateDrain` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.update_eligibility` | Time elapsed for `Node.UpdateEligibility` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.update_status` | Time elapsed for `Node.UpdateStatus` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_allocations.garbage_collect_all` | Time elapsed for `ClientAllocations.GarbageCollectAll` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_allocations.garbage_collect` | Time elapsed for `ClientAllocations.GarbageCollect` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_allocations.restart` | Time elapsed for `ClientAllocations.Restart` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_allocations.signal` | Time elapsed for `ClientAllocations.Signal` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_allocations.stats` | Time elapsed for `ClientAllocations.Stats` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_csi_controller.attach_volume` | Time elapsed for `Controller.AttachVolume` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_csi_controller.detach_volume` | Time elapsed for `Controller.DetachVolume` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_csi_controller.validate_volume` | Time elapsed for `Controller.ValidateVolume` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_csi_node.detach_volume` | Time elapsed for `Node.DetachVolume` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.allocations` | Time elapsed for `Deployment.Allocations` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.cancel` | Time elapsed for `Deployment.Cancel` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.fail` | Time elapsed for `Deployment.Fail` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.get_deployment` | Time elapsed for `Deployment.GetDeployment` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.list` | Time elapsed for `Deployment.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.pause` | Time elapsed for `Deployment.Pause` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.promote` | Time elapsed for `Deployment.Promote` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.reap` | Time elapsed for `Deployment.Reap` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.run` | Time elapsed for `Deployment.Run` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.set_alloc_health` | Time elapsed for `Deployment.SetAllocHealth` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.unblock` | Time elapsed for `Deployment.Unblock` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.ack` | Time elapsed for `Eval.Ack` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.allocations` | Time elapsed for `Eval.Allocations` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.create` | Time elapsed for `Eval.Create` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.dequeue` | Time elapsed for `Eval.Dequeue` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.get_eval` | Time elapsed for `Eval.GetEval` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.list` | Time elapsed for `Eval.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.nack` | Time elapsed for `Eval.Nack` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.reap` | Time elapsed for `Eval.Reap` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.reblock` | Time elapsed for `Eval.Reblock` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.update` | Time elapsed for `Eval.Update` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.file_system.list` | Time elapsed for `FileSystem.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.file_system.logs` | Time elapsed to establish `FileSystem.Logs` RPC | Nanoseconds | Summary | Host |
| `nomad.nomad.file_system.stat` | Time elapsed for `FileSystem.Stat` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.file_system.stream` | Time elapsed to establish `FileSystem.Stream` RPC | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.alloc_client_update` | Time elapsed to apply `AllocClientUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.alloc_update_desired_transition` | Time elapsed to apply `AllocUpdateDesiredTransition` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.alloc_update` | Time elapsed to apply `AllocUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_acl_policy_delete` | Time elapsed to apply `ApplyACLPolicyDelete` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_acl_policy_upsert` | Time elapsed to apply `ApplyACLPolicyUpsert` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_acl_token_bootstrap` | Time elapsed to apply `ApplyACLTokenBootstrap` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_acl_token_delete` | Time elapsed to apply `ApplyACLTokenDelete` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_acl_token_upsert` | Time elapsed to apply `ApplyACLTokenUpsert` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_csi_plugin_delete` | Time elapsed to apply `ApplyCSIPluginDelete` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_csi_volume_batch_claim` | Time elapsed to apply `ApplyCSIVolumeBatchClaim` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_csi_volume_claim` | Time elapsed to apply `ApplyCSIVolumeClaim` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_csi_volume_deregister` | Time elapsed to apply `ApplyCSIVolumeDeregister` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_csi_volume_register` | Time elapsed to apply `ApplyCSIVolumeRegister` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_deployment_alloc_health` | Time elapsed to apply `ApplyDeploymentAllocHealth` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_deployment_delete` | Time elapsed to apply `ApplyDeploymentDelete` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_deployment_promotion` | Time elapsed to apply `ApplyDeploymentPromotion` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_deployment_status_update` | Time elapsed to apply `ApplyDeploymentStatusUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_job_stability` | Time elapsed to apply `ApplyJobStability` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_namespace_delete` | Time elapsed to apply `ApplyNamespaceDelete` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_namespace_upsert` | Time elapsed to apply `ApplyNamespaceUpsert` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_plan_results` | Time elapsed to apply `ApplyPlanResults` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_scheduler_config` | Time elapsed to apply `ApplySchedulerConfig` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.autopilot` | Time elapsed to apply `Autopilot` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.batch_deregister_job` | Time elapsed to apply `BatchDeregisterJob` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.batch_deregister_node` | Time elapsed to apply `BatchDeregisterNode` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.batch_node_drain_update` | Time elapsed to apply `BatchNodeDrainUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.cluster_meta` | Time elapsed to apply `ClusterMeta` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.delete_eval` | Time elapsed to apply `DeleteEval` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.deregister_job` | Time elapsed to apply `DeregisterJob` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.deregister_node` | Time elapsed to apply `DeregisterNode` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.deregister_si_accessor` | Time elapsed to apply `DeregisterSITokenAccessor` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.deregister_vault_accessor` | Time elapsed to apply `DeregisterVaultAccessor` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.node_drain_update` | Time elapsed to apply `NodeDrainUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.node_eligibility_update` | Time elapsed to apply `NodeEligibilityUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.node_status_update` | Time elapsed to apply `NodeStatusUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.persist` | Time elapsed to apply `Persist` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.register_job` | Time elapsed to apply `RegisterJob` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.register_node` | Time elapsed to apply `RegisterNode` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.update_eval` | Time elapsed to apply `UpdateEval` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.upsert_node_events` | Time elapsed to apply `UpsertNodeEvents` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.upsert_scaling_event` | Time elapsed to apply `UpsertScalingEvent` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.upsert_si_accessor` | Time elapsed to apply `UpsertSITokenAccessors` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.upsert_vault_accessor` | Time elapsed to apply `UpsertVaultAccessor` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.job.allocations` | Time elapsed for `Job.Allocations` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.batch_deregister` | Time elapsed for `Job.BatchDeregister` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.deployments` | Time elapsed for `Job.Deployments` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.deregister` | Time elapsed for `Job.Deregister` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.dispatch` | Time elapsed for `Job.Dispatch` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.evaluate` | Time elapsed for `Job.Evaluate` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.evaluations` | Time elapsed for `Job.Evaluations` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.get_job_versions` | Time elapsed for `Job.GetJobVersions` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.get_job` | Time elapsed for `Job.GetJob` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.latest_deployment` | Time elapsed for `Job.LatestDeployment` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.list` | Time elapsed for `Job.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.plan` | Time elapsed for `Job.Plan` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.register` | Time elapsed for `Job.Register` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.revert` | Time elapsed for `Job.Revert` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.scale_status` | Time elapsed for `Job.ScaleStatus` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.scale` | Time elapsed for `Job.Scale` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.stable` | Time elapsed for `Job.Stable` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.validate` | Time elapsed for `Job.Validate` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job_summary.get_job_summary` | Time elapsed for `Job.Summary` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.leader.barrier` | Time elapsed to establish a raft barrier during leader transition | Nanoseconds | Summary | host |
| `nomad.nomad.leader.reconcileMember` | Time elapsed to reconcile a serf peer with state store | Nanoseconds | Summary | host |
| `nomad.nomad.leader.reconcile` | Time elapsed to reconcile all serf peers with state store | Nanoseconds | Summary | host |
| `nomad.nomad.namespace.delete_namespaces` | Time elapsed for `Namespace.DeleteNamespaces` | Nanoseconds | Summary | Host |
| `nomad.nomad.namespace.get_namespace` | Time elapsed for `Namespace.GetNamespace` | Nanoseconds | Summary | Host |
| `nomad.nomad.namespace.get_namespaces` | Time elapsed for `Namespace.GetNamespaces` | Nanoseconds | Summary | Host |
| `nomad.nomad.namespace.list_namespace` | Time elapsed for `Namespace.ListNamespaces` | Nanoseconds | Summary | Host |
| `nomad.nomad.namespace.upsert_namespaces` | Time elapsed for `Namespace.UpsertNamespaces` | Nanoseconds | Summary | Host |
| `nomad.nomad.periodic.force` | Time elapsed for `Periodic.Force` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.plan.apply` | Time elapsed to apply a plan | Nanoseconds | Summary | host |
| `nomad.nomad.plan.evaluate` | Time elapsed to evaluate a plan | Nanoseconds | Summary | host |
| `nomad.nomad.plan.queue_depth` | Count of evals in the plan queue | Integer | Gauge | host |
| `nomad.nomad.plan.submit` | Time elapsed for `Plan.Submit` RPC call | Nanoseconds | Summary | host |
| `nomad.nomad.plan.wait_for_index` | Time elapsed for the planner to obtain a snapshot | Nanoseconds | Summary | host |
| `nomad.nomad.plugin.delete` | Time elapsed for `CSIPlugin.Delete` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.plugin.get` | Time elapsed for `CSIPlugin.Get` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.plugin.list` | Time elapsed for `CSIPlugin.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.scaling.get_policy` | Time elapsed for `Scaling.GetPolicy` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.scaling.list_policies` | Time elapsed for `Scaling.ListPolicies` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.search.prefix_search` | Time elapsed for `Search.PrefixSearch` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.vault.create_token` | Time elapsed to create Vault token | Nanoseconds | Gauge | host |
| `nomad.nomad.vault.distributed_tokens_revoked` | Count of revoked tokens | Integer | Gauge | host |
| `nomad.nomad.vault.lookup_token` | Time elapsed to lookup Vault token | Nanoseconds | Gauge | host |
| `nomad.nomad.vault.renew_failed` | Count of failed attempts to renew Vault token | Integer | Gauge | host |
| `nomad.nomad.vault.renew` | Time elapsed to renew Vault token | Nanoseconds | Gauge | host |
| `nomad.nomad.vault.revoke_tokens` | Time elapsed to revoke Vault tokens | Nanoseconds | Gauge | host |
| `nomad.nomad.vault.token_ttl` | Time to live for Vault token | Integer | Gauge | host |
| `nomad.nomad.vault.undistributed_tokens_abandoned` | Count of abandoned tokens | Integer | Gauge | host |
| `nomad.nomad.volume.claim` | Time elapsed for `CSIVolume.Claim` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.volume.deregister` | Time elapsed for `CSIVolume.Deregister` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.volume.get` | Time elapsed for `CSIVolume.Get` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.volume.list` | Time elapsed for `CSIVolume.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.volume.register` | Time elapsed for `CSIVolume.Register` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.volume.unpublish` | Time elapsed for `CSIVolume.Unpublish` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.worker.create_eval` | Time elapsed for worker to create an eval | Nanoseconds | Summary | host |
| `nomad.nomad.worker.dequeue_eval` | Time elapsed for worker to dequeue an eval | Nanoseconds | Summary | host |
| `nomad.nomad.worker.invoke_scheduler_service` | Time elapsed for worker to invoke the scheduler | Nanoseconds | Summary | host |
| `nomad.nomad.worker.send_ack` | Time elapsed for worker to send acknowledgement | Nanoseconds | Summary | host |
| `nomad.nomad.worker.submit_plan` | Time elapsed for worker to submit plan | Nanoseconds | Summary | host |
| `nomad.nomad.worker.update_eval` | Time elapsed for worker to submit updated eval | Nanoseconds | Summary | host |
| `nomad.nomad.worker.wait_for_index` | Time elapsed for worker get snapshot | Nanoseconds | Summary | host |
| `nomad.raft.appliedIndex` | Current index applied to FSM | Integer | Gauge | host |
| `nomad.raft.barrier` | Count of blocking raft API calls | Integer | Counter | host |
| `nomad.raft.commitNumLogs` | Count of logs enqueued | Integer | Gauge | host |
| `nomad.raft.commitTime` | Time elapsed to commit writes | Nanoseconds | Summary | host |
| `nomad.raft.fsm.apply` | Time elapsed to apply write to FSM | Nanoseconds | Summary | host |
| `nomad.raft.fsm.enqueue` | Time elapsed to enqueue write to FSM | Nanoseconds | Summary | host |
| `nomad.raft.lastIndex` | Most recent index seen | Integer | Gauge | host |
| `nomad.raft.leader.dispatchLog` | Time elapsed to write log, mark in flight, and start replication | Nanoseconds | Summary | host |
| `nomad.raft.leader.dispatchNumLogs` | Count of logs dispatched | Integer | Gauge | host |
| `nomad.raft.replication.appendEntries` | Raft transaction commit time | ms / Raft Log Append | Timer | |
| `nomad.raft.state.candidate` | Count of entering candidate state | Integer | Gauge | host |
| `nomad.raft.state.follower` | Count of entering follower state | Integer | Gauge | host |
| `nomad.raft.state.leader` | Count of entering leader state | Integer | Gauge | host |
| `nomad.raft.transition.heartbeat_timeout` | Count of failing to heartbeat and starting election | Integer | Gauge | host |
| `nomad.raft.transition.leader_lease_timeout` | Count of stepping down as leader after losing quorum | Integer | Gauge | host |
| `nomad.runtime.free_count` | Count of objects freed from heap by go runtime GC | Integer | Gauge | host |
| `nomad.runtime.gc_pause_ns` | Go runtime GC pause times | Nanoseconds | Summary | host |
| `nomad.runtime.sys_bytes` | Go runtime GC metadata size | # of bytes | Gauge | host |
| `nomad.runtime.total_gc_pause_ns` | Total elapsed go runtime GC pause times | Nanoseconds | Gauge | host |
| `nomad.runtime.total_gc_runs` | Count of go runtime GC runs | Integer | Gauge | host |
| `nomad.serf.queue.Event` | Count of memberlist events received | Integer | Summary | host |
| `nomad.serf.queue.Intent` | Count of memberlist changes | Integer | Summary | host |
| `nomad.serf.queue.Query` | Count of memberlist queries | Integer | Summary | host |
| `nomad.state.snapshotIndex` | Current snapshot index | Integer | Gauge | host |
| Metric | Description | Unit | Type | Labels |
| ---------------------------------------------------- | ----------------------------------------------------------------- | -------------------- | ------- | ---------------------------- |
| `nomad.memberlist.gossip` | Time elapsed to broadcast gossip messages | Nanoseconds | Summary | host |
| `nomad.nomad.acl.bootstrap` | Time elapsed for `ACL.Bootstrap` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.delete_policies` | Time elapsed for `ACL.DeletePolicies` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.delete_tokens` | Time elapsed for `ACL.DeleteTokens` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.get_policies` | Time elapsed for `ACL.GetPolicies` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.get_policy` | Time elapsed for `ACL.GetPolicy` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.get_token` | Time elapsed for `ACL.GetToken` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.get_tokens` | Time elapsed for `ACL.GetTokens` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.list_policies` | Time elapsed for `ACL.ListPolicies` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.list_tokens` | Time elapsed for `ACL.ListTokens` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.resolve_token` | Time elapsed for `ACL.ResolveToken` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.upsert_policies` | Time elapsed for `ACL.UpsertPolicies` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.acl.upsert_tokens` | Time elapsed for `ACL.UpsertTokens` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.exec` | Time elapsed to establish alloc exec | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.get_alloc` | Time elapsed for `Alloc.GetAlloc` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.get_allocs` | Time elapsed for `Alloc.GetAllocs` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.list` | Time elapsed for `Alloc.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.stop` | Time elapsed for `Alloc.Stop` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.alloc.update_desired_transition` | Time elapsed for `Alloc.UpdateDesiredTransition` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.blocked_evals.cpu` | Amount of CPU shares requested by blocked evals | Integer | Gauge | datacenter, host, node_class |
| `nomad.nomad.blocked_evals.memory` | Amount of memory requested by blocked evals | Integer | Gauge | datacenter, host, node_class |
| `nomad.nomad.blocked_evals.job.cpu` | Amount of CPU shares requested by blocked evals of a job | Integer | Gauge | host, job, namespace |
| `nomad.nomad.blocked_evals.job.memory` | Amount of memory requested by blocked evals of a job | Integer | Gauge | host, job, namespace |
| `nomad.nomad.blocked_evals.total_blocked` | Count of evals in the blocked state | Integer | Gauge | host |
| `nomad.nomad.blocked_evals.total_escaped` | Count of evals that have escaped computed node classes | Integer | Gauge | host |
| `nomad.nomad.blocked_evals.total_quota_limit` | Count of blocked evals due to quota limits | Integer | Gauge | host |
| `nomad.nomad.broker.batch_ready` | Count of batch evals ready to be scheduled | Integer | Gauge | host |
| `nomad.nomad.broker.batch_unacked` | Count of unacknowledged batch evals | Integer | Gauge | host |
| `nomad.nomad.broker.service_ready` | Count of service evals ready to be scheduled | Integer | Gauge | host |
| `nomad.nomad.broker.service_unacked` | Count of unacknowledged service evals | Integer | Gauge | host |
| `nomad.nomad.broker.system_ready` | Count of system evals ready to be scheduled | Integer | Gauge | host |
| `nomad.nomad.broker.system_unacked` | Count of unacknowledged system evals | Integer | Gauge | host |
| `nomad.nomad.broker.total_ready` | Count of evals in the ready state | Integer | Gauge | host |
| `nomad.nomad.broker.total_waiting` | Count of evals in the waiting state | Integer | Gauge | host |
| `nomad.nomad.client.batch_deregister` | Time elapsed for `Node.BatchDeregister` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.deregister` | Time elapsed for `Node.Deregister` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.derive_si_token` | Time elapsed for `Node.DeriveSIToken` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.derive_vault_token` | Time elapsed for `Node.DeriveVaultToken` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.emit_events` | Time elapsed for `Node.EmitEvents` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.evaluate` | Time elapsed for `Node.Evaluate` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.get_allocs` | Time elapsed for `Node.GetAllocs` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.get_client_allocs` | Time elapsed for `Node.GetClientAllocs` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.get_node` | Time elapsed for `Node.GetNode` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.list` | Time elapsed for `Node.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.register` | Time elapsed for `Node.Register` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.stats` | Time elapsed for `Client.Stats` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.update_alloc` | Time elapsed for `Node.UpdateAlloc` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.update_drain` | Time elapsed for `Node.UpdateDrain` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.update_eligibility` | Time elapsed for `Node.UpdateEligibility` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client.update_status` | Time elapsed for `Node.UpdateStatus` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_allocations.garbage_collect_all` | Time elapsed for `ClientAllocations.GarbageCollectAll` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_allocations.garbage_collect` | Time elapsed for `ClientAllocations.GarbageCollect` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_allocations.restart` | Time elapsed for `ClientAllocations.Restart` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_allocations.signal` | Time elapsed for `ClientAllocations.Signal` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_allocations.stats` | Time elapsed for `ClientAllocations.Stats` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_csi_controller.attach_volume` | Time elapsed for `Controller.AttachVolume` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_csi_controller.detach_volume` | Time elapsed for `Controller.DetachVolume` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_csi_controller.validate_volume` | Time elapsed for `Controller.ValidateVolume` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.client_csi_node.detach_volume` | Time elapsed for `Node.DetachVolume` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.allocations` | Time elapsed for `Deployment.Allocations` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.cancel` | Time elapsed for `Deployment.Cancel` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.fail` | Time elapsed for `Deployment.Fail` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.get_deployment` | Time elapsed for `Deployment.GetDeployment` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.list` | Time elapsed for `Deployment.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.pause` | Time elapsed for `Deployment.Pause` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.promote` | Time elapsed for `Deployment.Promote` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.reap` | Time elapsed for `Deployment.Reap` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.run` | Time elapsed for `Deployment.Run` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.set_alloc_health` | Time elapsed for `Deployment.SetAllocHealth` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.deployment.unblock` | Time elapsed for `Deployment.Unblock` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.ack` | Time elapsed for `Eval.Ack` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.allocations` | Time elapsed for `Eval.Allocations` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.create` | Time elapsed for `Eval.Create` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.dequeue` | Time elapsed for `Eval.Dequeue` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.get_eval` | Time elapsed for `Eval.GetEval` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.list` | Time elapsed for `Eval.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.nack` | Time elapsed for `Eval.Nack` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.reap` | Time elapsed for `Eval.Reap` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.reblock` | Time elapsed for `Eval.Reblock` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.eval.update` | Time elapsed for `Eval.Update` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.file_system.list` | Time elapsed for `FileSystem.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.file_system.logs` | Time elapsed to establish `FileSystem.Logs` RPC | Nanoseconds | Summary | Host |
| `nomad.nomad.file_system.stat` | Time elapsed for `FileSystem.Stat` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.file_system.stream` | Time elapsed to establish `FileSystem.Stream` RPC | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.alloc_client_update` | Time elapsed to apply `AllocClientUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.alloc_update_desired_transition` | Time elapsed to apply `AllocUpdateDesiredTransition` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.alloc_update` | Time elapsed to apply `AllocUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_acl_policy_delete` | Time elapsed to apply `ApplyACLPolicyDelete` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_acl_policy_upsert` | Time elapsed to apply `ApplyACLPolicyUpsert` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_acl_token_bootstrap` | Time elapsed to apply `ApplyACLTokenBootstrap` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_acl_token_delete` | Time elapsed to apply `ApplyACLTokenDelete` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_acl_token_upsert` | Time elapsed to apply `ApplyACLTokenUpsert` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_csi_plugin_delete` | Time elapsed to apply `ApplyCSIPluginDelete` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_csi_volume_batch_claim` | Time elapsed to apply `ApplyCSIVolumeBatchClaim` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_csi_volume_claim` | Time elapsed to apply `ApplyCSIVolumeClaim` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_csi_volume_deregister` | Time elapsed to apply `ApplyCSIVolumeDeregister` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_csi_volume_register` | Time elapsed to apply `ApplyCSIVolumeRegister` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_deployment_alloc_health` | Time elapsed to apply `ApplyDeploymentAllocHealth` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_deployment_delete` | Time elapsed to apply `ApplyDeploymentDelete` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_deployment_promotion` | Time elapsed to apply `ApplyDeploymentPromotion` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_deployment_status_update` | Time elapsed to apply `ApplyDeploymentStatusUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_job_stability` | Time elapsed to apply `ApplyJobStability` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_namespace_delete` | Time elapsed to apply `ApplyNamespaceDelete` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_namespace_upsert` | Time elapsed to apply `ApplyNamespaceUpsert` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_plan_results` | Time elapsed to apply `ApplyPlanResults` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.apply_scheduler_config` | Time elapsed to apply `ApplySchedulerConfig` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.autopilot` | Time elapsed to apply `Autopilot` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.batch_deregister_job` | Time elapsed to apply `BatchDeregisterJob` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.batch_deregister_node` | Time elapsed to apply `BatchDeregisterNode` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.batch_node_drain_update` | Time elapsed to apply `BatchNodeDrainUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.cluster_meta` | Time elapsed to apply `ClusterMeta` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.delete_eval` | Time elapsed to apply `DeleteEval` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.deregister_job` | Time elapsed to apply `DeregisterJob` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.deregister_node` | Time elapsed to apply `DeregisterNode` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.deregister_si_accessor` | Time elapsed to apply `DeregisterSITokenAccessor` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.deregister_vault_accessor` | Time elapsed to apply `DeregisterVaultAccessor` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.node_drain_update` | Time elapsed to apply `NodeDrainUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.node_eligibility_update` | Time elapsed to apply `NodeEligibilityUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.node_status_update` | Time elapsed to apply `NodeStatusUpdate` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.persist` | Time elapsed to apply `Persist` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.register_job` | Time elapsed to apply `RegisterJob` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.register_node` | Time elapsed to apply `RegisterNode` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.update_eval` | Time elapsed to apply `UpdateEval` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.upsert_node_events` | Time elapsed to apply `UpsertNodeEvents` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.upsert_scaling_event` | Time elapsed to apply `UpsertScalingEvent` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.upsert_si_accessor` | Time elapsed to apply `UpsertSITokenAccessors` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.fsm.upsert_vault_accessor` | Time elapsed to apply `UpsertVaultAccessor` raft entry | Nanoseconds | Summary | Host |
| `nomad.nomad.job.allocations` | Time elapsed for `Job.Allocations` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.batch_deregister` | Time elapsed for `Job.BatchDeregister` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.deployments` | Time elapsed for `Job.Deployments` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.deregister` | Time elapsed for `Job.Deregister` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.dispatch` | Time elapsed for `Job.Dispatch` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.evaluate` | Time elapsed for `Job.Evaluate` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.evaluations` | Time elapsed for `Job.Evaluations` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.get_job_versions` | Time elapsed for `Job.GetJobVersions` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.get_job` | Time elapsed for `Job.GetJob` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.latest_deployment` | Time elapsed for `Job.LatestDeployment` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.list` | Time elapsed for `Job.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.plan` | Time elapsed for `Job.Plan` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.register` | Time elapsed for `Job.Register` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.revert` | Time elapsed for `Job.Revert` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.scale_status` | Time elapsed for `Job.ScaleStatus` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.scale` | Time elapsed for `Job.Scale` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.stable` | Time elapsed for `Job.Stable` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job.validate` | Time elapsed for `Job.Validate` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.job_summary.get_job_summary` | Time elapsed for `Job.Summary` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.leader.barrier` | Time elapsed to establish a raft barrier during leader transition | Nanoseconds | Summary | host |
| `nomad.nomad.leader.reconcileMember` | Time elapsed to reconcile a serf peer with state store | Nanoseconds | Summary | host |
| `nomad.nomad.leader.reconcile` | Time elapsed to reconcile all serf peers with state store | Nanoseconds | Summary | host |
| `nomad.nomad.namespace.delete_namespaces` | Time elapsed for `Namespace.DeleteNamespaces` | Nanoseconds | Summary | Host |
| `nomad.nomad.namespace.get_namespace` | Time elapsed for `Namespace.GetNamespace` | Nanoseconds | Summary | Host |
| `nomad.nomad.namespace.get_namespaces` | Time elapsed for `Namespace.GetNamespaces` | Nanoseconds | Summary | Host |
| `nomad.nomad.namespace.list_namespace` | Time elapsed for `Namespace.ListNamespaces` | Nanoseconds | Summary | Host |
| `nomad.nomad.namespace.upsert_namespaces` | Time elapsed for `Namespace.UpsertNamespaces` | Nanoseconds | Summary | Host |
| `nomad.nomad.periodic.force` | Time elapsed for `Periodic.Force` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.plan.apply` | Time elapsed to apply a plan | Nanoseconds | Summary | host |
| `nomad.nomad.plan.evaluate` | Time elapsed to evaluate a plan | Nanoseconds | Summary | host |
| `nomad.nomad.plan.queue_depth` | Count of evals in the plan queue | Integer | Gauge | host |
| `nomad.nomad.plan.submit` | Time elapsed for `Plan.Submit` RPC call | Nanoseconds | Summary | host |
| `nomad.nomad.plan.wait_for_index` | Time elapsed for the planner to obtain a snapshot | Nanoseconds | Summary | host |
| `nomad.nomad.plugin.delete` | Time elapsed for `CSIPlugin.Delete` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.plugin.get` | Time elapsed for `CSIPlugin.Get` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.plugin.list` | Time elapsed for `CSIPlugin.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.scaling.get_policy` | Time elapsed for `Scaling.GetPolicy` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.scaling.list_policies` | Time elapsed for `Scaling.ListPolicies` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.search.prefix_search` | Time elapsed for `Search.PrefixSearch` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.vault.create_token` | Time elapsed to create Vault token | Nanoseconds | Gauge | host |
| `nomad.nomad.vault.distributed_tokens_revoked` | Count of revoked tokens | Integer | Gauge | host |
| `nomad.nomad.vault.lookup_token` | Time elapsed to lookup Vault token | Nanoseconds | Gauge | host |
| `nomad.nomad.vault.renew_failed` | Count of failed attempts to renew Vault token | Integer | Gauge | host |
| `nomad.nomad.vault.renew` | Time elapsed to renew Vault token | Nanoseconds | Gauge | host |
| `nomad.nomad.vault.revoke_tokens` | Time elapsed to revoke Vault tokens | Nanoseconds | Gauge | host |
| `nomad.nomad.vault.token_ttl` | Time to live for Vault token | Integer | Gauge | host |
| `nomad.nomad.vault.undistributed_tokens_abandoned` | Count of abandoned tokens | Integer | Gauge | host |
| `nomad.nomad.volume.claim` | Time elapsed for `CSIVolume.Claim` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.volume.deregister` | Time elapsed for `CSIVolume.Deregister` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.volume.get` | Time elapsed for `CSIVolume.Get` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.volume.list` | Time elapsed for `CSIVolume.List` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.volume.register` | Time elapsed for `CSIVolume.Register` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.volume.unpublish` | Time elapsed for `CSIVolume.Unpublish` RPC call | Nanoseconds | Summary | Host |
| `nomad.nomad.worker.create_eval` | Time elapsed for worker to create an eval | Nanoseconds | Summary | host |
| `nomad.nomad.worker.dequeue_eval` | Time elapsed for worker to dequeue an eval | Nanoseconds | Summary | host |
| `nomad.nomad.worker.invoke_scheduler_service` | Time elapsed for worker to invoke the scheduler | Nanoseconds | Summary | host |
| `nomad.nomad.worker.send_ack` | Time elapsed for worker to send acknowledgement | Nanoseconds | Summary | host |
| `nomad.nomad.worker.submit_plan` | Time elapsed for worker to submit plan | Nanoseconds | Summary | host |
| `nomad.nomad.worker.update_eval` | Time elapsed for worker to submit updated eval | Nanoseconds | Summary | host |
| `nomad.nomad.worker.wait_for_index` | Time elapsed for worker get snapshot | Nanoseconds | Summary | host |
| `nomad.raft.appliedIndex` | Current index applied to FSM | Integer | Gauge | host |
| `nomad.raft.barrier` | Count of blocking raft API calls | Integer | Counter | host |
| `nomad.raft.commitNumLogs` | Count of logs enqueued | Integer | Gauge | host |
| `nomad.raft.commitTime` | Time elapsed to commit writes | Nanoseconds | Summary | host |
| `nomad.raft.fsm.apply` | Time elapsed to apply write to FSM | Nanoseconds | Summary | host |
| `nomad.raft.fsm.enqueue` | Time elapsed to enqueue write to FSM | Nanoseconds | Summary | host |
| `nomad.raft.lastIndex` | Most recent index seen | Integer | Gauge | host |
| `nomad.raft.leader.dispatchLog` | Time elapsed to write log, mark in flight, and start replication | Nanoseconds | Summary | host |
| `nomad.raft.leader.dispatchNumLogs` | Count of logs dispatched | Integer | Gauge | host |
| `nomad.raft.replication.appendEntries` | Raft transaction commit time | ms / Raft Log Append | Timer | |
| `nomad.raft.state.candidate` | Count of entering candidate state | Integer | Gauge | host |
| `nomad.raft.state.follower` | Count of entering follower state | Integer | Gauge | host |
| `nomad.raft.state.leader` | Count of entering leader state | Integer | Gauge | host |
| `nomad.raft.transition.heartbeat_timeout` | Count of failing to heartbeat and starting election | Integer | Gauge | host |
| `nomad.raft.transition.leader_lease_timeout` | Count of stepping down as leader after losing quorum | Integer | Gauge | host |
| `nomad.runtime.free_count` | Count of objects freed from heap by go runtime GC | Integer | Gauge | host |
| `nomad.runtime.gc_pause_ns` | Go runtime GC pause times | Nanoseconds | Summary | host |
| `nomad.runtime.sys_bytes` | Go runtime GC metadata size | # of bytes | Gauge | host |
| `nomad.runtime.total_gc_pause_ns` | Total elapsed go runtime GC pause times | Nanoseconds | Gauge | host |
| `nomad.runtime.total_gc_runs` | Count of go runtime GC runs | Integer | Gauge | host |
| `nomad.serf.queue.Event` | Count of memberlist events received | Integer | Summary | host |
| `nomad.serf.queue.Intent` | Count of memberlist changes | Integer | Summary | host |
| `nomad.serf.queue.Query` | Count of memberlist queries | Integer | Summary | host |
| `nomad.state.snapshotIndex` | Current snapshot index | Integer | Gauge | host |
[tagged-metrics]: /docs/telemetry/metrics#tagged-metrics