feat(metrics): enable some metrics; push to graphite

* -trace flag controls whether or not to enable metrics-gathering
  and the /debug/* HTTP endpoints
* -graphite-host flag controls where metrics should be sent
* timer.ae.handle metric tracks execution time of AppendEntriesRequest
This commit is contained in:
Brian Waldon 2014-01-17 16:22:21 -08:00
parent 3e7c2dff96
commit 7eaad5c8e0
6 changed files with 67 additions and 7 deletions

25
etcd.go
View File

@ -19,9 +19,11 @@ package main
import (
"fmt"
"os"
"runtime"
"time"
"github.com/coreos/etcd/log"
"github.com/coreos/etcd/metrics"
"github.com/coreos/etcd/server"
"github.com/coreos/etcd/store"
"github.com/coreos/raft"
@ -81,6 +83,21 @@ func main() {
log.Fatal("Peer TLS:", err)
}
var mbName string
if config.Trace() {
mbName = config.MetricsBucketName()
runtime.SetBlockProfileRate(1)
}
mb := metrics.NewBucket(mbName)
if config.GraphiteHost != "" {
err := mb.Publish(config.GraphiteHost)
if err != nil {
panic(err)
}
}
// Create etcd key-value store and registry.
store := store.New()
registry := server.NewRegistry(store)
@ -88,16 +105,20 @@ func main() {
// Create peer server.
heartbeatTimeout := time.Duration(config.Peer.HeartbeatTimeout) * time.Millisecond
electionTimeout := time.Duration(config.Peer.ElectionTimeout) * time.Millisecond
ps := server.NewPeerServer(info.Name, config.DataDir, info.RaftURL, info.RaftListenHost, &peerTLSConfig, &info.RaftTLS, registry, store, config.SnapshotCount, heartbeatTimeout, electionTimeout)
ps := server.NewPeerServer(info.Name, config.DataDir, info.RaftURL, info.RaftListenHost, &peerTLSConfig, &info.RaftTLS, registry, store, config.SnapshotCount, heartbeatTimeout, electionTimeout, &mb)
ps.MaxClusterSize = config.MaxClusterSize
ps.RetryTimes = config.MaxRetryAttempts
// Create client server.
s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &tlsConfig, &info.EtcdTLS, ps, registry, store)
s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &tlsConfig, &info.EtcdTLS, ps, registry, store, &mb)
if err := s.AllowOrigins(config.CorsOrigins); err != nil {
panic(err)
}
if config.Trace() {
s.EnableTracing()
}
ps.SetServer(s)
// Run peer server in separate thread while the client server blocks.

View File

@ -77,6 +77,8 @@ type Config struct {
HeartbeatTimeout int `toml:"heartbeat_timeout" env:"ETCD_PEER_HEARTBEAT_TIMEOUT"`
ElectionTimeout int `toml:"election_timeout" env:"ETCD_PEER_ELECTION_TIMEOUT"`
}
strTrace string `toml:"trace" env:"ETCD_TRACE"`
GraphiteHost string `toml:"graphite_host" env:"ETCD_GRAPHITE_HOST"`
}
// NewConfig returns a Config initialized with default values.
@ -247,6 +249,9 @@ func (c *Config) LoadFlags(arguments []string) error {
f.IntVar(&c.SnapshotCount, "snapshot-count", c.SnapshotCount, "")
f.StringVar(&c.CPUProfileFile, "cpuprofile", "", "")
f.StringVar(&c.strTrace, "trace", "", "")
f.StringVar(&c.GraphiteHost, "graphite-host", "", "")
// BEGIN IGNORED FLAGS
f.StringVar(&path, "config", "", "")
// BEGIN IGNORED FLAGS
@ -453,6 +458,17 @@ func (c *Config) PeerTLSConfig() (TLSConfig, error) {
return c.PeerTLSInfo().Config()
}
// MetricsBucketName generates the name that should be used for a
// corresponding MetricsBucket object
func (c *Config) MetricsBucketName() string {
return fmt.Sprintf("etcd.%s", c.Name)
}
// Trace determines if any trace-level information should be emitted
func (c *Config) Trace() bool {
return c.strTrace == "*"
}
// sanitizeURL will cleanup a host string in the format hostname[:port] and
// attach a schema.
func sanitizeURL(host string, defaultScheme string) (string, error) {

View File

@ -15,6 +15,7 @@ import (
etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/etcd/log"
"github.com/coreos/etcd/metrics"
"github.com/coreos/etcd/store"
"github.com/coreos/raft"
"github.com/gorilla/mux"
@ -47,6 +48,8 @@ type PeerServer struct {
closeChan chan bool
timeoutThresholdChan chan interface{}
metrics *metrics.Bucket
}
// TODO: find a good policy to do snapshot
@ -62,7 +65,8 @@ type snapshotConf struct {
snapshotThr uint64
}
func NewPeerServer(name string, path string, url string, bindAddr string, tlsConf *TLSConfig, tlsInfo *TLSInfo, registry *Registry, store store.Store, snapshotCount int, heartbeatTimeout, electionTimeout time.Duration) *PeerServer {
func NewPeerServer(name string, path string, url string, bindAddr string, tlsConf *TLSConfig, tlsInfo *TLSInfo, registry *Registry, store store.Store, snapshotCount int, heartbeatTimeout, electionTimeout time.Duration, mb *metrics.Bucket) *PeerServer {
s := &PeerServer{
name: name,
url: url,
@ -89,6 +93,8 @@ func NewPeerServer(name string, path string, url string, bindAddr string, tlsCon
ElectionTimeout: electionTimeout,
timeoutThresholdChan: make(chan interface{}, 1),
metrics: mb,
}
// Create transporter for raft

View File

@ -4,6 +4,7 @@ import (
"encoding/json"
"net/http"
"strconv"
"time"
etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/etcd/log"
@ -49,6 +50,7 @@ func (ps *PeerServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request)
// Response to append entries request
func (ps *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) {
start := time.Now()
aereq := &raft.AppendEntriesRequest{}
if _, err := aereq.Decode(req.Body); err != nil {
@ -78,6 +80,8 @@ func (ps *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.
http.Error(w, "", http.StatusInternalServerError)
return
}
(*ps.metrics).Timer("timer.appendentries.handle").UpdateSince(start)
}
// Response to recover from snapshot request

View File

@ -12,6 +12,7 @@ import (
etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/etcd/log"
"github.com/coreos/etcd/metrics"
"github.com/coreos/etcd/mod"
"github.com/coreos/etcd/server/v1"
"github.com/coreos/etcd/server/v2"
@ -34,10 +35,11 @@ type Server struct {
tlsInfo *TLSInfo
router *mux.Router
corsHandler *corsHandler
metrics *metrics.Bucket
}
// Creates a new Server.
func New(name string, urlStr string, bindAddr string, tlsConf *TLSConfig, tlsInfo *TLSInfo, peerServer *PeerServer, registry *Registry, store store.Store) *Server {
func New(name string, urlStr string, bindAddr string, tlsConf *TLSConfig, tlsInfo *TLSInfo, peerServer *PeerServer, registry *Registry, store store.Store, mb *metrics.Bucket) *Server {
r := mux.NewRouter()
cors := &corsHandler{router: r}
@ -56,6 +58,7 @@ func New(name string, urlStr string, bindAddr string, tlsConf *TLSConfig, tlsInf
peerServer: peerServer,
router: r,
corsHandler: cors,
metrics: mb,
}
// Install the routes.
@ -63,11 +66,14 @@ func New(name string, urlStr string, bindAddr string, tlsConf *TLSConfig, tlsInf
s.installV1()
s.installV2()
s.installMod()
s.installDebug()
return s
}
func (s *Server) EnableTracing() {
s.installDebug()
}
// The current state of the server in the cluster.
func (s *Server) State() string {
return s.peerServer.RaftServer().State()
@ -141,6 +147,7 @@ func (s *Server) installMod() {
}
func (s *Server) installDebug() {
s.handleFunc("/debug/metrics", s.GetMetricsHandler).Methods("GET")
s.router.HandleFunc("/debug/pprof", pprof.Index)
s.router.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline)
s.router.HandleFunc("/debug/pprof/profile", pprof.Profile)
@ -402,3 +409,9 @@ func (s *Server) SpeedTestHandler(w http.ResponseWriter, req *http.Request) erro
w.Write([]byte("speed test success"))
return nil
}
// Retrieves metrics from bucket
func (s *Server) GetMetricsHandler(w http.ResponseWriter, req *http.Request) error {
(*s.metrics).Dump(w)
return nil
}

View File

@ -26,9 +26,9 @@ func RunServer(f func(*server.Server)) {
store := store.New()
registry := server.NewRegistry(store)
ps := server.NewPeerServer(testName, path, "http://"+testRaftURL, testRaftURL, &server.TLSConfig{Scheme: "http"}, &server.TLSInfo{}, registry, store, testSnapshotCount, testHeartbeatTimeout, testElectionTimeout)
ps := server.NewPeerServer(testName, path, "http://"+testRaftURL, testRaftURL, &server.TLSConfig{Scheme: "http"}, &server.TLSInfo{}, registry, store, testSnapshotCount, testHeartbeatTimeout, testElectionTimeout, nil)
ps.MaxClusterSize = 9
s := server.New(testName, "http://"+testClientURL, testClientURL, &server.TLSConfig{Scheme: "http"}, &server.TLSInfo{}, ps, registry, store)
s := server.New(testName, "http://"+testClientURL, testClientURL, &server.TLSConfig{Scheme: "http"}, &server.TLSInfo{}, ps, registry, store, nil)
ps.SetServer(s)
// Start up peer server.