mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #9762 from gyuho/rrr
*: document all histogram metrics, add etcd_snap_fsync_duration_seconds
This commit is contained in:
commit
83fd1c3faa
@ -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)
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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)))
|
||||
}
|
||||
|
@ -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),
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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)
|
||||
|
@ -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),
|
||||
|
@ -17,25 +17,42 @@ package snap
|
||||
import "github.com/prometheus/client_golang/prometheus"
|
||||
|
||||
var (
|
||||
// TODO: save_fsync latency?
|
||||
saveDurations = 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),
|
||||
})
|
||||
|
||||
marshallingDurations = 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.",
|
||||
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),
|
||||
})
|
||||
|
||||
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.",
|
||||
|
||||
// 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),
|
||||
})
|
||||
|
||||
snapFsyncSec = prometheus.NewHistogram(prometheus.HistogramOpts{
|
||||
Namespace: "etcd",
|
||||
Subsystem: "snap",
|
||||
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)
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
||||
|
@ -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) {
|
||||
|
@ -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 {
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
||||
|
@ -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):
|
||||
|
@ -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.
|
||||
|
@ -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) {
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user