mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
*: add pending/failed proposal metrics
This commit is contained in:
parent
aa503f84d5
commit
598fa7a10e
@ -26,7 +26,8 @@ All these metrics are prefixed with `etcd_server_`
|
||||
| leader_changes_seen_total | The number of leader changes seen. | Counter |
|
||||
| proposals_committed_total | The total number of consensus proposals committed. | Gauge |
|
||||
| proposals_applied_total | The total number of consensus proposals applied. | Gauge |
|
||||
|
||||
| proposals_pending | The current number of pending proposals. | Gauge |
|
||||
| proposals_failed_total | The total number of failed proposals seen. | Counter |
|
||||
|
||||
`has_leader` indicates whether the member has a leader. If a member does not have a leader, it is
|
||||
totally unavailable. If all the members in the cluster do not have any leader, the entire cluster
|
||||
@ -38,6 +39,10 @@ is totally unavailable.
|
||||
|
||||
`proposals_applied_total` records the total number of consensus proposals applied. The etcd server applies every committed proposal asynchronously. The difference between `proposals_committed_total` and `proposals_applied_total` should usually be small (within a few thousands even under high load). If the difference between them continues to rise, it indicates that the etcd server is overloaded. This might happen when applying expensive queries like heavy range queries or large txn operations.
|
||||
|
||||
`proposals_pending` indicates how many proposals are queued to commit. Rising pending proposals suggests there is a high client load or the member cannot commit proposals.
|
||||
|
||||
`proposals_failed_total` are normally related to two issues: temporary failures related to a leader election or longer downtime caused by a loss of quorum in the cluster.
|
||||
|
||||
### disk
|
||||
|
||||
These metrics describe the status of the disk operations.
|
||||
@ -102,19 +107,6 @@ Example Prometheus queries that may be useful from these metrics (across all etc
|
||||
|
||||
The metrics under the `etcd_debugging` prefix are for debugging. They are very implementation dependent and volatile. They might be changed or removed without any warning in new etcd releases. Some of the metrics might be moved to the `etcd` prefix when they become more stable.
|
||||
|
||||
### etcdserver
|
||||
|
||||
| Name | Description | Type |
|
||||
|-----------------------------------------|--------------------------------------------------|-----------|
|
||||
| proposal_duration_seconds | The latency distributions of committing proposal | Histogram |
|
||||
| proposals_pending | The current number of pending proposals | Gauge |
|
||||
| proposals_failed_total | The total number of failed proposals | Counter |
|
||||
|
||||
[Proposal][glossary-proposal] duration (`proposal_duration_seconds`) provides a proposal commit latency histogram. The reported latency reflects network and disk IO delays in etcd.
|
||||
|
||||
Proposals pending (`proposals_pending`) indicates how many proposals are queued for commit. Rising pending proposals suggests there is a high client load or the cluster is unstable.
|
||||
|
||||
Failed proposals (`proposals_failed_total`) are normally related to two issues: temporary failures related to a leader election or longer duration downtime caused by a loss of quorum in the cluster.
|
||||
|
||||
### snapshot
|
||||
|
||||
|
@ -22,30 +22,6 @@ import (
|
||||
)
|
||||
|
||||
var (
|
||||
// TODO: with label in v3?
|
||||
proposeDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
|
||||
Namespace: "etcd_debugging",
|
||||
Subsystem: "server",
|
||||
Name: "proposal_duration_seconds",
|
||||
Help: "The latency distributions of committing proposal.",
|
||||
Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
|
||||
})
|
||||
proposePending = prometheus.NewGauge(prometheus.GaugeOpts{
|
||||
Namespace: "etcd_debugging",
|
||||
Subsystem: "server",
|
||||
Name: "proposals_pending",
|
||||
Help: "The current number of pending proposals.",
|
||||
})
|
||||
// This is number of proposal failed in client's view.
|
||||
// The proposal might be later got committed in raft.
|
||||
proposeFailed = prometheus.NewCounter(prometheus.CounterOpts{
|
||||
Namespace: "etcd_debugging",
|
||||
Subsystem: "server",
|
||||
Name: "proposals_failed_total",
|
||||
Help: "The total number of failed proposals.",
|
||||
})
|
||||
|
||||
// stable metrics for monitoring
|
||||
hasLeader = prometheus.NewGauge(prometheus.GaugeOpts{
|
||||
Namespace: "etcd",
|
||||
Subsystem: "server",
|
||||
@ -70,16 +46,27 @@ var (
|
||||
Name: "proposals_applied_total",
|
||||
Help: "The total number of consensus proposals applied.",
|
||||
})
|
||||
proposalsPending = prometheus.NewGauge(prometheus.GaugeOpts{
|
||||
Namespace: "etcd",
|
||||
Subsystem: "server",
|
||||
Name: "proposals_pending",
|
||||
Help: "The current number of pending proposals to commit.",
|
||||
})
|
||||
proposalsFailed = prometheus.NewGauge(prometheus.GaugeOpts{
|
||||
Namespace: "etcd",
|
||||
Subsystem: "server",
|
||||
Name: "proposals_failed_total",
|
||||
Help: "The total number of failed proposals seen.",
|
||||
})
|
||||
)
|
||||
|
||||
func init() {
|
||||
prometheus.MustRegister(proposeDurations)
|
||||
prometheus.MustRegister(proposePending)
|
||||
prometheus.MustRegister(proposeFailed)
|
||||
prometheus.MustRegister(hasLeader)
|
||||
prometheus.MustRegister(leaderChanges)
|
||||
prometheus.MustRegister(proposalsCommitted)
|
||||
prometheus.MustRegister(proposalsApplied)
|
||||
prometheus.MustRegister(proposalsPending)
|
||||
prometheus.MustRegister(proposalsFailed)
|
||||
}
|
||||
|
||||
func monitorFileDescriptor(done <-chan struct{}) {
|
||||
|
@ -55,21 +55,17 @@ func (a *v2apiStore) processRaftRequest(ctx context.Context, r *pb.Request) (Res
|
||||
}
|
||||
ch := a.s.w.Register(r.ID)
|
||||
|
||||
// TODO: benchmark the cost of time.Now()
|
||||
// might be sampling?
|
||||
start := time.Now()
|
||||
a.s.r.Propose(ctx, data)
|
||||
|
||||
proposePending.Inc()
|
||||
defer proposePending.Dec()
|
||||
proposalsPending.Inc()
|
||||
defer proposalsPending.Dec()
|
||||
|
||||
select {
|
||||
case x := <-ch:
|
||||
proposeDurations.Observe(float64(time.Since(start)) / float64(time.Second))
|
||||
resp := x.(Response)
|
||||
return resp, resp.err
|
||||
case <-ctx.Done():
|
||||
proposeFailed.Inc()
|
||||
proposalsFailed.Inc()
|
||||
a.s.w.Trigger(r.ID, nil) // GC wait
|
||||
return Response{}, a.s.parseProposeCtxErr(ctx.Err(), start)
|
||||
case <-a.s.done:
|
||||
|
@ -522,14 +522,18 @@ func (s *EtcdServer) processInternalRaftRequest(ctx context.Context, r pb.Intern
|
||||
cctx, cancel := context.WithTimeout(ctx, maxV3RequestTimeout)
|
||||
defer cancel()
|
||||
|
||||
start := time.Now()
|
||||
s.r.Propose(cctx, data)
|
||||
proposalsPending.Inc()
|
||||
defer proposalsPending.Dec()
|
||||
|
||||
select {
|
||||
case x := <-ch:
|
||||
return x.(*applyResult), nil
|
||||
case <-cctx.Done():
|
||||
proposalsFailed.Inc()
|
||||
s.w.Trigger(id, nil) // GC wait
|
||||
return nil, cctx.Err()
|
||||
return nil, s.parseProposeCtxErr(cctx.Err(), start)
|
||||
case <-s.done:
|
||||
return nil, ErrStopped
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user