Selaa lähdekoodia

Merge pull request #9762 from gyuho/rrr

*: document all histogram metrics, add etcd_snap_fsync_duration_seconds
Gyuho Lee 7 vuotta sitten
vanhempi
commit
83fd1c3faa

+ 26 - 3
etcdserver/api/rafthttp/metrics.go

@@ -17,6 +17,24 @@ package rafthttp
 import "github.com/prometheus/client_golang/prometheus"
 
 var (
+	activePeers = prometheus.NewGaugeVec(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "active_peers",
+		Help:      "The current number of active peer connections.",
+	},
+		[]string{"Local", "Remote"},
+	)
+
+	disconnectedPeers = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "disconnected_peers_total",
+		Help:      "The total number of disconnected peers.",
+	},
+		[]string{"Local", "Remote"},
+	)
+
 	sentBytes = prometheus.NewCounterVec(prometheus.CounterOpts{
 		Namespace: "etcd",
 		Subsystem: "network",
@@ -53,21 +71,26 @@ var (
 		[]string{"From"},
 	)
 
-	rtts = prometheus.NewHistogramVec(prometheus.HistogramOpts{
+	rttSec = prometheus.NewHistogramVec(prometheus.HistogramOpts{
 		Namespace: "etcd",
 		Subsystem: "network",
 		Name:      "peer_round_trip_time_seconds",
 		Help:      "Round-Trip-Time histogram between peers.",
-		Buckets:   prometheus.ExponentialBuckets(0.0001, 2, 14),
+
+		// lowest bucket start of upper bound 0.0001 sec (0.1 ms) with factor 2
+		// highest bucket start of 0.0001 sec * 2^15 == 3.2768 sec
+		Buckets: prometheus.ExponentialBuckets(0.0001, 2, 16),
 	},
 		[]string{"To"},
 	)
 )
 
 func init() {
+	prometheus.MustRegister(activePeers)
+	prometheus.MustRegister(disconnectedPeers)
 	prometheus.MustRegister(sentBytes)
 	prometheus.MustRegister(receivedBytes)
 	prometheus.MustRegister(sentFailures)
 	prometheus.MustRegister(recvFailures)
-	prometheus.MustRegister(rtts)
+	prometheus.MustRegister(rttSec)
 }

+ 1 - 1
etcdserver/api/rafthttp/peer.go

@@ -137,7 +137,7 @@ func startPeer(t *Transport, urls types.URLs, peerID types.ID, fs *stats.Followe
 		}
 	}()
 
-	status := newPeerStatus(t.Logger, peerID)
+	status := newPeerStatus(t.Logger, t.ID, peerID)
 	picker := newURLPicker(urls)
 	errorc := t.ErrorC
 	r := t.Raft

+ 9 - 2
etcdserver/api/rafthttp/peer_status.go

@@ -32,14 +32,15 @@ type failureType struct {
 
 type peerStatus struct {
 	lg     *zap.Logger
+	local  types.ID
 	id     types.ID
 	mu     sync.Mutex // protect variables below
 	active bool
 	since  time.Time
 }
 
-func newPeerStatus(lg *zap.Logger, id types.ID) *peerStatus {
-	return &peerStatus{lg: lg, id: id}
+func newPeerStatus(lg *zap.Logger, local, id types.ID) *peerStatus {
+	return &peerStatus{lg: lg, local: local, id: id}
 }
 
 func (s *peerStatus) activate() {
@@ -53,6 +54,8 @@ func (s *peerStatus) activate() {
 		}
 		s.active = true
 		s.since = time.Now()
+
+		activePeers.WithLabelValues(s.local.String(), s.id.String()).Inc()
 	}
 }
 
@@ -69,8 +72,12 @@ func (s *peerStatus) deactivate(failure failureType, reason string) {
 		}
 		s.active = false
 		s.since = time.Time{}
+
+		activePeers.WithLabelValues(s.local.String(), s.id.String()).Dec()
+		disconnectedPeers.WithLabelValues(s.local.String(), s.id.String()).Inc()
 		return
 	}
+
 	if s.lg != nil {
 		s.lg.Debug("peer deactivated again", zap.String("peer-id", s.id.String()), zap.Error(errors.New(msg)))
 	}

+ 1 - 1
etcdserver/api/rafthttp/pipeline_test.go

@@ -303,7 +303,7 @@ func startTestPipeline(tr *Transport, picker *urlPicker) *pipeline {
 		peerID:        types.ID(1),
 		tr:            tr,
 		picker:        picker,
-		status:        newPeerStatus(zap.NewExample(), types.ID(1)),
+		status:        newPeerStatus(zap.NewExample(), tr.ID, types.ID(1)),
 		raft:          &fakeRaft{},
 		followerStats: &stats.FollowerStats{},
 		errorc:        make(chan error, 1),

+ 5 - 3
etcdserver/api/rafthttp/probing_status.go

@@ -44,9 +44,10 @@ func addPeerToProber(lg *zap.Logger, p probing.Prober, id string, us []string) {
 		} else {
 			plog.Errorf("failed to add peer %s into prober", id)
 		}
-	} else {
-		go monitorProbingStatus(lg, s, id)
+		return
 	}
+
+	go monitorProbingStatus(lg, s, id)
 }
 
 func monitorProbingStatus(lg *zap.Logger, s probing.Status, id string) {
@@ -83,7 +84,8 @@ func monitorProbingStatus(lg *zap.Logger, s probing.Status, id string) {
 					plog.Warningf("the clock difference against peer %s is too high [%v > %v]", id, s.ClockDiff(), time.Second)
 				}
 			}
-			rtts.WithLabelValues(id).Observe(s.SRTT().Seconds())
+			rttSec.WithLabelValues(id).Observe(s.SRTT().Seconds())
+
 		case <-s.StopNotify():
 			return
 		}

+ 1 - 1
etcdserver/api/rafthttp/remote.go

@@ -31,7 +31,7 @@ type remote struct {
 
 func startRemote(tr *Transport, urls types.URLs, id types.ID) *remote {
 	picker := newURLPicker(urls)
-	status := newPeerStatus(tr.Logger, id)
+	status := newPeerStatus(tr.Logger, tr.ID, id)
 	pipeline := &pipeline{
 		peerID: id,
 		tr:     tr,

+ 1 - 1
etcdserver/api/rafthttp/snapshot_test.go

@@ -109,7 +109,7 @@ func testSnapshotSend(t *testing.T, sm *snap.Message) (bool, []os.FileInfo) {
 	defer srv.Close()
 
 	picker := mustNewURLPicker(t, []string{srv.URL})
-	snapsend := newSnapshotSender(tr, picker, types.ID(1), newPeerStatus(zap.NewExample(), types.ID(1)))
+	snapsend := newSnapshotSender(tr, picker, types.ID(1), newPeerStatus(zap.NewExample(), types.ID(0), types.ID(1)))
 	defer snapsend.stop()
 
 	snapsend.send(*sm)

+ 5 - 5
etcdserver/api/rafthttp/stream_test.go

@@ -41,7 +41,7 @@ import (
 // to streamWriter. After that, streamWriter can use it to send messages
 // continuously, and closes it when stopped.
 func TestStreamWriterAttachOutgoingConn(t *testing.T) {
-	sw := startStreamWriter(zap.NewExample(), types.ID(0), types.ID(1), newPeerStatus(zap.NewExample(), types.ID(1)), &stats.FollowerStats{}, &fakeRaft{})
+	sw := startStreamWriter(zap.NewExample(), types.ID(0), types.ID(1), newPeerStatus(zap.NewExample(), types.ID(0), types.ID(1)), &stats.FollowerStats{}, &fakeRaft{})
 	// the expected initial state of streamWriter is not working
 	if _, ok := sw.writec(); ok {
 		t.Errorf("initial working status = %v, want false", ok)
@@ -93,7 +93,7 @@ func TestStreamWriterAttachOutgoingConn(t *testing.T) {
 // TestStreamWriterAttachBadOutgoingConn tests that streamWriter with bad
 // outgoingConn will close the outgoingConn and fall back to non-working status.
 func TestStreamWriterAttachBadOutgoingConn(t *testing.T) {
-	sw := startStreamWriter(zap.NewExample(), types.ID(0), types.ID(1), newPeerStatus(zap.NewExample(), types.ID(1)), &stats.FollowerStats{}, &fakeRaft{})
+	sw := startStreamWriter(zap.NewExample(), types.ID(0), types.ID(1), newPeerStatus(zap.NewExample(), types.ID(0), types.ID(1)), &stats.FollowerStats{}, &fakeRaft{})
 	defer sw.stop()
 	wfc := newFakeWriteFlushCloser(errors.New("blah"))
 	sw.attach(&outgoingConn{t: streamTypeMessage, Writer: wfc, Flusher: wfc, Closer: wfc})
@@ -197,7 +197,7 @@ func TestStreamReaderStopOnDial(t *testing.T) {
 		picker: mustNewURLPicker(t, []string{"http://localhost:2380"}),
 		errorc: make(chan error, 1),
 		typ:    streamTypeMessage,
-		status: newPeerStatus(zap.NewExample(), types.ID(2)),
+		status: newPeerStatus(zap.NewExample(), types.ID(1), types.ID(2)),
 		rl:     rate.NewLimiter(rate.Every(100*time.Millisecond), 1),
 	}
 	tr.onResp = func() {
@@ -304,7 +304,7 @@ func TestStream(t *testing.T) {
 		srv := httptest.NewServer(h)
 		defer srv.Close()
 
-		sw := startStreamWriter(zap.NewExample(), types.ID(0), types.ID(1), newPeerStatus(zap.NewExample(), types.ID(1)), &stats.FollowerStats{}, &fakeRaft{})
+		sw := startStreamWriter(zap.NewExample(), types.ID(0), types.ID(1), newPeerStatus(zap.NewExample(), types.ID(0), types.ID(1)), &stats.FollowerStats{}, &fakeRaft{})
 		defer sw.stop()
 		h.sw = sw
 
@@ -316,7 +316,7 @@ func TestStream(t *testing.T) {
 			typ:    tt.t,
 			tr:     tr,
 			picker: picker,
-			status: newPeerStatus(zap.NewExample(), types.ID(2)),
+			status: newPeerStatus(zap.NewExample(), types.ID(0), types.ID(2)),
 			recvc:  recvc,
 			propc:  propc,
 			rl:     rate.NewLimiter(rate.Every(100*time.Millisecond), 1),

+ 27 - 10
etcdserver/api/snap/metrics.go

@@ -17,25 +17,42 @@ package snap
 import "github.com/prometheus/client_golang/prometheus"
 
 var (
-	// TODO: save_fsync latency?
-	saveDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
+	snapMarshallingSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "snap",
+		Name:      "save_marshalling_duration_seconds",
+		Help:      "The marshalling cost distributions of save called by snapshot.",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
+	})
+
+	snapSaveSec = prometheus.NewHistogram(prometheus.HistogramOpts{
 		Namespace: "etcd_debugging",
 		Subsystem: "snap",
 		Name:      "save_total_duration_seconds",
 		Help:      "The total latency distributions of save called by snapshot.",
-		Buckets:   prometheus.ExponentialBuckets(0.001, 2, 14),
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
 	})
 
-	marshallingDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
-		Namespace: "etcd_debugging",
+	snapFsyncSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd",
 		Subsystem: "snap",
-		Name:      "save_marshalling_duration_seconds",
-		Help:      "The marshalling cost distributions of save called by snapshot.",
-		Buckets:   prometheus.ExponentialBuckets(0.001, 2, 14),
+		Name:      "fsync_duration_seconds",
+		Help:      "The latency distributions of fsync called by snap.",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
 	})
 )
 
 func init() {
-	prometheus.MustRegister(saveDurations)
-	prometheus.MustRegister(marshallingDurations)
+	prometheus.MustRegister(snapMarshallingSec)
+	prometheus.MustRegister(snapSaveSec)
+	prometheus.MustRegister(snapFsyncSec)
 }

+ 6 - 3
etcdserver/api/snap/snapshotter.go

@@ -81,11 +81,14 @@ func (s *Snapshotter) save(snapshot *raftpb.Snapshot) error {
 	if err != nil {
 		return err
 	}
-
-	marshallingDurations.Observe(float64(time.Since(start)) / float64(time.Second))
+	snapMarshallingSec.Observe(time.Since(start).Seconds())
 
 	spath := filepath.Join(s.dir, fname)
+
+	fsyncStart := time.Now()
 	err = pioutil.WriteAndSyncFile(spath, d, 0666)
+	snapFsyncSec.Observe(time.Since(fsyncStart).Seconds())
+
 	if err != nil {
 		if s.lg != nil {
 			s.lg.Warn("failed to write a snap file", zap.String("path", spath), zap.Error(err))
@@ -101,7 +104,7 @@ func (s *Snapshotter) save(snapshot *raftpb.Snapshot) error {
 		return err
 	}
 
-	saveDurations.Observe(float64(time.Since(start)) / float64(time.Second))
+	snapSaveSec.Observe(time.Since(start).Seconds())
 	return nil
 }
 

+ 8 - 4
etcdserver/api/v2http/metrics.go

@@ -23,6 +23,7 @@ import (
 	"github.com/coreos/etcd/etcdserver/api/v2error"
 	"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
 	"github.com/coreos/etcd/etcdserver/etcdserverpb"
+
 	"github.com/prometheus/client_golang/prometheus"
 )
 
@@ -43,20 +44,23 @@ var (
 			Help:      "Counter of handle failures of requests (non-watches), by method (GET/PUT etc.) and code (400, 500 etc.).",
 		}, []string{"method", "code"})
 
-	successfulEventsHandlingTime = prometheus.NewHistogramVec(
+	successfulEventsHandlingSec = prometheus.NewHistogramVec(
 		prometheus.HistogramOpts{
 			Namespace: "etcd",
 			Subsystem: "http",
 			Name:      "successful_duration_seconds",
 			Help:      "Bucketed histogram of processing time (s) of successfully handled requests (non-watches), by method (GET/PUT etc.).",
-			Buckets:   prometheus.ExponentialBuckets(0.0005, 2, 13),
+
+			// lowest bucket start of upper bound 0.0005 sec (0.5 ms) with factor 2
+			// highest bucket start of 0.0005 sec * 2^12 == 2.048 sec
+			Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13),
 		}, []string{"method"})
 )
 
 func init() {
 	prometheus.MustRegister(incomingEvents)
 	prometheus.MustRegister(failedEvents)
-	prometheus.MustRegister(successfulEventsHandlingTime)
+	prometheus.MustRegister(successfulEventsHandlingSec)
 }
 
 func reportRequestReceived(request etcdserverpb.Request) {
@@ -65,7 +69,7 @@ func reportRequestReceived(request etcdserverpb.Request) {
 
 func reportRequestCompleted(request etcdserverpb.Request, startTime time.Time) {
 	method := methodFromRequest(request)
-	successfulEventsHandlingTime.WithLabelValues(method).Observe(time.Since(startTime).Seconds())
+	successfulEventsHandlingSec.WithLabelValues(method).Observe(time.Since(startTime).Seconds())
 }
 
 func reportRequestFailed(request etcdserverpb.Request, err error) {

+ 3 - 2
mvcc/backend/backend.go

@@ -230,8 +230,9 @@ func (b *backend) Snapshot() Snapshot {
 				} else {
 					plog.Warningf("snapshotting is taking more than %v seconds to finish transferring %v MB [started at %v]", time.Since(start).Seconds(), float64(dbBytes)/float64(1024*1014), start)
 				}
+
 			case <-stopc:
-				snapshotDurations.Observe(time.Since(start).Seconds())
+				snapshotTransferSec.Observe(time.Since(start).Seconds())
 				return
 			}
 		}
@@ -416,7 +417,7 @@ func (b *backend) defrag() error {
 	atomic.StoreInt64(&b.sizeInUse, size-(int64(db.Stats().FreePageN)*int64(db.Info().PageSize)))
 
 	took := time.Since(now)
-	defragDurations.Observe(took.Seconds())
+	defragSec.Observe(took.Seconds())
 
 	size2, sizeInUse2 := b.Size(), b.SizeInUse()
 	if b.lg != nil {

+ 1 - 1
mvcc/backend/batch_tx.go

@@ -220,7 +220,7 @@ func (t *batchTx) commit(stop bool) {
 		err := t.tx.Commit()
 		// gofail: var afterCommit struct{}
 
-		commitDurations.Observe(time.Since(start).Seconds())
+		commitSec.Observe(time.Since(start).Seconds())
 		atomic.AddInt64(&t.backend.commits, 1)
 
 		t.pending = 0

+ 6 - 6
mvcc/backend/metrics.go

@@ -17,7 +17,7 @@ package backend
 import "github.com/prometheus/client_golang/prometheus"
 
 var (
-	commitDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
+	commitSec = prometheus.NewHistogram(prometheus.HistogramOpts{
 		Namespace: "etcd",
 		Subsystem: "disk",
 		Name:      "backend_commit_duration_seconds",
@@ -28,7 +28,7 @@ var (
 		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
 	})
 
-	defragDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
+	defragSec = prometheus.NewHistogram(prometheus.HistogramOpts{
 		Namespace: "etcd",
 		Subsystem: "disk",
 		Name:      "backend_defrag_duration_seconds",
@@ -40,7 +40,7 @@ var (
 		Buckets: prometheus.ExponentialBuckets(.1, 2, 13),
 	})
 
-	snapshotDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
+	snapshotTransferSec = prometheus.NewHistogram(prometheus.HistogramOpts{
 		Namespace: "etcd",
 		Subsystem: "disk",
 		Name:      "backend_snapshot_duration_seconds",
@@ -53,7 +53,7 @@ var (
 )
 
 func init() {
-	prometheus.MustRegister(commitDurations)
-	prometheus.MustRegister(defragDurations)
-	prometheus.MustRegister(snapshotDurations)
+	prometheus.MustRegister(commitSec)
+	prometheus.MustRegister(defragSec)
+	prometheus.MustRegister(snapshotTransferSec)
 }

+ 3 - 3
mvcc/kvstore.go

@@ -168,7 +168,7 @@ func (s *store) Hash() (hash uint32, revision int64, err error) {
 	s.b.ForceCommit()
 	h, err := s.b.Hash(DefaultIgnores)
 
-	hashDurations.Observe(time.Since(start).Seconds())
+	hashSec.Observe(time.Since(start).Seconds())
 	return h, s.currentRev, err
 }
 
@@ -221,7 +221,7 @@ func (s *store) HashByRev(rev int64) (hash uint32, currentRev int64, compactRev
 	})
 	hash = h.Sum32()
 
-	hashRevDurations.Observe(time.Since(start).Seconds())
+	hashRevSec.Observe(time.Since(start).Seconds())
 	return hash, currentRev, compactRev, err
 }
 
@@ -274,7 +274,7 @@ func (s *store) Compact(rev int64) (<-chan struct{}, error) {
 
 	s.fifoSched.Schedule(j)
 
-	indexCompactionPauseDurations.Observe(float64(time.Since(start) / time.Millisecond))
+	indexCompactionPauseMs.Observe(float64(time.Since(start) / time.Millisecond))
 	return ch, nil
 }
 

+ 2 - 2
mvcc/kvstore_compaction.go

@@ -23,7 +23,7 @@ import (
 
 func (s *store) scheduleCompaction(compactMainRev int64, keep map[revision]struct{}) bool {
 	totalStart := time.Now()
-	defer dbCompactionTotalDurations.Observe(float64(time.Since(totalStart) / time.Millisecond))
+	defer dbCompactionTotalMs.Observe(float64(time.Since(totalStart) / time.Millisecond))
 	keyCompactions := 0
 	defer func() { dbCompactionKeysCounter.Add(float64(keyCompactions)) }()
 
@@ -68,7 +68,7 @@ func (s *store) scheduleCompaction(compactMainRev int64, keep map[revision]struc
 		// update last
 		revToBytes(revision{main: rev.main, sub: rev.sub + 1}, last)
 		tx.Unlock()
-		dbCompactionPauseDurations.Observe(float64(time.Since(start) / time.Millisecond))
+		dbCompactionPauseMs.Observe(float64(time.Since(start) / time.Millisecond))
 
 		select {
 		case <-time.After(100 * time.Millisecond):

+ 21 - 15
mvcc/metrics.go

@@ -101,33 +101,39 @@ var (
 			Help:      "Total number of pending events to be sent.",
 		})
 
-	indexCompactionPauseDurations = prometheus.NewHistogram(
+	indexCompactionPauseMs = prometheus.NewHistogram(
 		prometheus.HistogramOpts{
 			Namespace: "etcd_debugging",
 			Subsystem: "mvcc",
 			Name:      "index_compaction_pause_duration_milliseconds",
 			Help:      "Bucketed histogram of index compaction pause duration.",
-			// 0.5ms -> 1second
-			Buckets: prometheus.ExponentialBuckets(0.5, 2, 12),
+
+			// lowest bucket start of upper bound 0.5 ms with factor 2
+			// highest bucket start of 0.5 ms * 2^13 == 4.096 sec
+			Buckets: prometheus.ExponentialBuckets(0.5, 2, 14),
 		})
 
-	dbCompactionPauseDurations = prometheus.NewHistogram(
+	dbCompactionPauseMs = prometheus.NewHistogram(
 		prometheus.HistogramOpts{
 			Namespace: "etcd_debugging",
 			Subsystem: "mvcc",
 			Name:      "db_compaction_pause_duration_milliseconds",
 			Help:      "Bucketed histogram of db compaction pause duration.",
-			// 1ms -> 4second
+
+			// lowest bucket start of upper bound 1 ms with factor 2
+			// highest bucket start of 1 ms * 2^12 == 4.096 sec
 			Buckets: prometheus.ExponentialBuckets(1, 2, 13),
 		})
 
-	dbCompactionTotalDurations = prometheus.NewHistogram(
+	dbCompactionTotalMs = prometheus.NewHistogram(
 		prometheus.HistogramOpts{
 			Namespace: "etcd_debugging",
 			Subsystem: "mvcc",
 			Name:      "db_compaction_total_duration_milliseconds",
 			Help:      "Bucketed histogram of db compaction total duration.",
-			// 100ms -> 800second
+
+			// lowest bucket start of upper bound 100 ms with factor 2
+			// highest bucket start of 100 ms * 2^13 == 8.192 sec
 			Buckets: prometheus.ExponentialBuckets(100, 2, 14),
 		})
 
@@ -169,9 +175,9 @@ var (
 	)
 	// overridden by mvcc initialization
 	reportDbTotalSizeInUseInBytesMu sync.RWMutex
-	reportDbTotalSizeInUseInBytes   func() float64 = func() float64 { return 0 }
+	reportDbTotalSizeInUseInBytes   = func() float64 { return 0 }
 
-	hashDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
+	hashSec = prometheus.NewHistogram(prometheus.HistogramOpts{
 		Namespace: "etcd",
 		Subsystem: "mvcc",
 		Name:      "hash_duration_seconds",
@@ -183,7 +189,7 @@ var (
 		Buckets: prometheus.ExponentialBuckets(.01, 2, 15),
 	})
 
-	hashRevDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
+	hashRevSec = prometheus.NewHistogram(prometheus.HistogramOpts{
 		Namespace: "etcd",
 		Subsystem: "mvcc",
 		Name:      "hash_rev_duration_seconds",
@@ -207,14 +213,14 @@ func init() {
 	prometheus.MustRegister(slowWatcherGauge)
 	prometheus.MustRegister(totalEventsCounter)
 	prometheus.MustRegister(pendingEventsGauge)
-	prometheus.MustRegister(indexCompactionPauseDurations)
-	prometheus.MustRegister(dbCompactionPauseDurations)
-	prometheus.MustRegister(dbCompactionTotalDurations)
+	prometheus.MustRegister(indexCompactionPauseMs)
+	prometheus.MustRegister(dbCompactionPauseMs)
+	prometheus.MustRegister(dbCompactionTotalMs)
 	prometheus.MustRegister(dbCompactionKeysCounter)
 	prometheus.MustRegister(dbTotalSize)
 	prometheus.MustRegister(dbTotalSizeInUse)
-	prometheus.MustRegister(hashDurations)
-	prometheus.MustRegister(hashRevDurations)
+	prometheus.MustRegister(hashSec)
+	prometheus.MustRegister(hashRevSec)
 }
 
 // ReportEventReceived reports that an event is received.

+ 7 - 5
proxy/httpproxy/metrics.go

@@ -47,13 +47,15 @@ var (
 			Help:      "Counter of requests dropped on the proxy.",
 		}, []string{"method", "proxying_error"})
 
-	requestsHandlingTime = prometheus.NewHistogramVec(
+	requestsHandlingSec = prometheus.NewHistogramVec(
 		prometheus.HistogramOpts{
 			Namespace: "etcd",
 			Subsystem: "proxy",
 			Name:      "handling_duration_seconds",
-			Help: "Bucketed histogram of handling time of successful events (non-watches), by method " +
-				"(GET/PUT etc.).",
+			Help:      "Bucketed histogram of handling time of successful events (non-watches), by method (GET/PUT etc.).",
+
+			// lowest bucket start of upper bound 0.0005 sec (0.5 ms) with factor 2
+			// highest bucket start of 0.0005 sec * 2^12 == 2.048 sec
 			Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13),
 		}, []string{"method"})
 )
@@ -70,7 +72,7 @@ func init() {
 	prometheus.MustRegister(requestsIncoming)
 	prometheus.MustRegister(requestsHandled)
 	prometheus.MustRegister(requestsDropped)
-	prometheus.MustRegister(requestsHandlingTime)
+	prometheus.MustRegister(requestsHandlingSec)
 }
 
 func reportIncomingRequest(request *http.Request) {
@@ -80,7 +82,7 @@ func reportIncomingRequest(request *http.Request) {
 func reportRequestHandled(request *http.Request, response *http.Response, startTime time.Time) {
 	method := request.Method
 	requestsHandled.WithLabelValues(method, strconv.Itoa(response.StatusCode)).Inc()
-	requestsHandlingTime.WithLabelValues(method).Observe(time.Since(startTime).Seconds())
+	requestsHandlingSec.WithLabelValues(method).Observe(time.Since(startTime).Seconds())
 }
 
 func reportRequestDropped(request *http.Request, err forwardingError) {

+ 7 - 4
wal/metrics.go

@@ -17,15 +17,18 @@ package wal
 import "github.com/prometheus/client_golang/prometheus"
 
 var (
-	syncDurations = prometheus.NewHistogram(prometheus.HistogramOpts{
+	walFsyncSec = prometheus.NewHistogram(prometheus.HistogramOpts{
 		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),
+		Help:      "The latency distributions of fsync called by WAL.",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
 	})
 )
 
 func init() {
-	prometheus.MustRegister(syncDurations)
+	prometheus.MustRegister(walFsyncSec)
 }

+ 1 - 1
wal/wal.go

@@ -590,7 +590,7 @@ func (w *WAL) sync() error {
 			plog.Warningf("sync duration of %v, expected less than %v", took, warnSyncDuration)
 		}
 	}
-	syncDurations.Observe(took.Seconds())
+	walFsyncSec.Observe(took.Seconds())
 
 	return err
 }