Merge pull request #5309 from xiang90/d_metrics

*: add disk operation metrics for monitoring
release-3.0
Xiang Li 2016-05-11 10:18:39 -07:00
commit 00b660cc53
5 changed files with 55 additions and 23 deletions

View File

@ -35,6 +35,23 @@ is totally unavailable.
`proposals_committed_total` records the total number of consensus proposals committed. This gauge should increase over time if the cluster is healthy. Several healthy members of an etcd cluster may have different total committed proposals at once. This discrepancy may be due to recovering from peers after starting, lagging behind the leader, or being the leader and therefore having the most commits. It is important to monitor this metric across all the members in the cluster; a consistently large lag between a single member and its leader indicates that member is slow or unhealthy.
### disk
These metrics describe the status of the disk operations.
All these metrics are prefixed with `etcd_disk_`.
| Name | Description | Type |
|------------------------------------|-------------------------------------------------------|-----------|
| wal_fsync_duration_seconds | The latency distributions of fsync called by wal | Histogram |
| backend_commit_duration_seconds | The latency distributions of commit called by backend.| Histogram |
A `wal_fsync` is called when etcd persists its log entries to disk before applying them.
A `backend_commit` is called when etcd commits an incremental snapshot of its most recent changes to disk.
High disk operation latencies (`wal_fsync_duration_seconds` or `backend_commit_duration_seconds`) often indicate disk issues. It may cause high request latency or make the cluster unstable.
### network
These metrics describe the status of the network.
@ -96,15 +113,6 @@ Proposals pending (`proposals_pending`) indicates how many proposals are queued
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.
### wal
| Name | Description | Type |
|------------------------------------|--------------------------------------------------|-----------|
| fsync_duration_seconds | The latency distributions of fsync called by wal | Histogram |
| last_index_saved | The index of the last entry saved by wal | Gauge |
Abnormally high fsync duration (`fsync_duration_seconds`) indicates disk issues and might cause the cluster to be unstable.
### snapshot
| Name | Description | Type |

View File

@ -19,6 +19,7 @@ import (
"log"
"sync"
"sync/atomic"
"time"
"github.com/boltdb/bolt"
)
@ -165,7 +166,9 @@ func (t *batchTx) commit(stop bool) {
atomic.StoreInt64(&t.backend.size, t.tx.Size())
return
}
start := time.Now()
err = t.tx.Commit()
commitDurations.Observe(time.Since(start).Seconds())
atomic.AddInt64(&t.backend.commits, 1)
t.pending = 0

31
mvcc/backend/metrics.go Normal file
View File

@ -0,0 +1,31 @@
// Copyright 2016 CoreOS, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package backend
import "github.com/prometheus/client_golang/prometheus"
var (
commitDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
Namespace: "etcd",
Subsystem: "disk",
Name: "backend_commit_duration_seconds",
Help: "The latency distributions of commit called by backend.",
Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
})
)
func init() {
prometheus.MustRegister(commitDurations)
}

View File

@ -18,21 +18,14 @@ import "github.com/prometheus/client_golang/prometheus"
var (
syncDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
Namespace: "etcd_debugging",
Subsystem: "wal",
Name: "fsync_duration_seconds",
Namespace: "etcd",
Subsystem: "disk",
Name: "wal_fsync_duration_seconds",
Help: "The latency distributions of fsync called by wal.",
Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
})
lastIndexSaved = prometheus.NewGauge(prometheus.GaugeOpts{
Namespace: "etcd_debugging",
Subsystem: "wal",
Name: "last_index_saved",
Help: "The index of the last entry saved by wal.",
})
)
func init() {
prometheus.MustRegister(syncDurations)
prometheus.MustRegister(lastIndexSaved)
}

View File

@ -307,7 +307,6 @@ func (w *WAL) ReadAll() (metadata []byte, state raftpb.HardState, ents []raftpb.
// create encoder (chain crc with the decoder), enable appending
_, err = w.tail().Seek(w.decoder.lastOffset(), os.SEEK_SET)
w.encoder = newEncoder(w.tail(), w.decoder.lastCRC())
lastIndexSaved.Set(float64(w.enti))
}
w.decoder = nil
@ -390,7 +389,7 @@ func (w *WAL) sync() error {
}
start := time.Now()
err := fileutil.Fdatasync(w.tail().File)
syncDurations.Observe(float64(time.Since(start)) / float64(time.Second))
syncDurations.Observe(time.Since(start).Seconds())
return err
}
@ -471,7 +470,6 @@ func (w *WAL) saveEntry(e *raftpb.Entry) error {
return err
}
w.enti = e.Index
lastIndexSaved.Set(float64(w.enti))
return nil
}
@ -534,7 +532,6 @@ func (w *WAL) SaveSnapshot(e walpb.Snapshot) error {
if w.enti < e.Index {
w.enti = e.Index
}
lastIndexSaved.Set(float64(w.enti))
return w.sync()
}