Merge pull request #3093 from mwitkow-io/feature/httpd_metrics

add `events` metrics in etcdhttp.
release-2.3
Xiang Li 2015-10-01 12:10:58 -07:00
commit 715fdfb669
5 changed files with 140 additions and 46 deletions

View File

@ -30,45 +30,6 @@ Pending proposal (`pending_proposal_total`) gives you an idea about how many pro
Failed proposals (`proposal_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.
### store
These metrics describe the accesses into the data store of etcd members that exist in the cluster. They
are useful to count what kind of actions are taken by users. It is also useful to see and whether all etcd members
"see" the same set of data mutations, and whether reads and watches (which are local) are equally distributed.
All these metrics are prefixed with `etcd_store_`.
| Name | Description | Type |
|---------------------------|------------------------------------------------------------------------------------------|--------------------|
| reads_total | Total number of reads from store, should differ among etcd members (local reads). | Counter(action) |
| writes_total | Total number of writes to store, should be same among all etcd members. | Counter(action) |
| reads_failed_total | Number of failed reads from store (e.g. key missing) on local reads. | Counter(action) |
| writes_failed_total | Number of failed writes to store (e.g. failed compare and swap). | Counter(action) |
| expires_total | Total number of expired keys (due to TTL).   | Counter |
| watch_requests_totals | Total number of incoming watch requests to this etcd member (local watches). | Counter |
| watchers | Current count of active watchers on this etcd member. | Gauge |
Both `reads_total` and `writes_total` count both successful and failed requests. `reads_failed_total` and
`writes_failed_total` count failed requests. A lot of failed writes indicate possible contentions on keys (e.g. when
doing `compareAndSet`), and read failures indicate that some clients try to access keys that don't exist.
Example Prometheus queries that may be useful from these metrics (across all etcd members):
* `sum(rate(etcd_store_reads_total{job="etcd"}[1m])) by (action)`
`max(rate(etcd_store_writes_total{job="etcd"}[1m])) by (action)`
Rate of reads and writes by action, across all servers across a time window of `1m`. The reason why `max` is used
for writes as opposed to `sum` for reads is because all of etcd nodes in the cluster apply all writes to their stores.
Shows the rate of successful readonly/write queries across all servers, across a time window of `1m`.
* `sum(rate(etcd_store_watch_requests_total{job="etcd"}[1m]))`
Shows rate of new watch requests per second. Likely driven by how often watched keys change.
* `sum(etcd_store_watchers{job="etcd"})`
Number of active watchers across all etcd servers.
### wal
| Name | Description | Type |
@ -78,6 +39,38 @@ Example Prometheus queries that may be useful from these metrics (across all etc
Abnormally high fsync duration (`fsync_durations_microseconds`) indicates disk issues and might cause the cluster to be unstable.
### http requests
These metrics describe the serving of requests (non-watch events) served by etcd members in non-proxy mode: total
incoming requests, request failures and processing latency (inc. raft rounds for storage). They are useful for tracking
user-generated traffic hitting the etcd cluster .
All these metrics are prefixed with `etcd_http_`
| Name | Description | Type |
|--------------------------------|-----------------------------------------------------------------------------------------|--------------------|
| received_total | Total number of events after parsing and auth. | Counter(method) |
| failed_total | Total number of failed events.   | Counter(method,error) |
| successful_duration_second | Bucketed handling times of the requests, including raft rounds for writes. | Histogram(method) |
Example Prometheus queries that may be useful from these metrics (across all etcd members):
* `sum(rate(etcd_http_failed_total{job="etcd"}[1m]) by (method) / sum(rate(etcd_http_events_received_total{job="etcd"})[1m]) by (method)`
Shows the fraction of events that failed by HTTP method across all members, across a time window of `1m`.
* `sum(rate(etcd_http_received_total{job="etcd",method="GET})[1m]) by (method)`
`sum(rate(etcd_http_received_total{job="etcd",method~="GET})[1m]) by (method)`
Shows the rate of successful readonly/write queries across all servers, across a time window of `1m`.
* `histogram_quantile(0.9, sum(increase(etcd_http_successful_processing_seconds{job="etcd",method="GET"}[5m]) ) by (le))`
`histogram_quantile(0.9, sum(increase(etcd_http_successful_processing_seconds{job="etcd",method!="GET"}[5m]) ) by (le))`
Show the 0.90-tile latency (in seconds) of read/write (respectively) event handling across all members, with a window of `5m`.
### snapshot
| Name | Description | Type |

View File

@ -143,7 +143,7 @@ func (e Error) toJsonString() string {
return string(b)
}
func (e Error) statusCode() int {
func (e Error) StatusCode() int {
status, ok := errorStatus[e.ErrorCode]
if !ok {
status = http.StatusBadRequest
@ -154,6 +154,6 @@ func (e Error) statusCode() int {
func (e Error) WriteTo(w http.ResponseWriter) {
w.Header().Add("X-Etcd-Index", fmt.Sprint(e.Index))
w.Header().Set("Content-Type", "application/json")
w.WriteHeader(e.statusCode())
w.WriteHeader(e.StatusCode())
fmt.Fprintln(w, e.toJsonString())
}

View File

@ -28,8 +28,8 @@ func TestErrorWriteTo(t *testing.T) {
rr := httptest.NewRecorder()
err.WriteTo(rr)
if err.statusCode() != rr.Code {
t.Errorf("HTTP status code %d, want %d", rr.Code, err.statusCode())
if err.StatusCode() != rr.Code {
t.Errorf("HTTP status code %d, want %d", rr.Code, err.StatusCode())
}
gbody := strings.TrimSuffix(rr.Body.String(), "\n")

View File

@ -128,8 +128,9 @@ func (h *keysHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
ctx, cancel := context.WithTimeout(context.Background(), h.timeout)
defer cancel()
rr, err := parseKeyRequest(r, clockwork.NewRealClock())
clock := clockwork.NewRealClock()
startTime := clock.Now()
rr, err := parseKeyRequest(r, clock)
if err != nil {
writeKeyError(w, err)
return
@ -139,11 +140,14 @@ func (h *keysHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
writeKeyNoAuth(w)
return
}
if !rr.Wait {
reportRequestReceived(rr)
}
resp, err := h.server.Do(ctx, rr)
if err != nil {
err = trimErrorPrefix(err, etcdserver.StoreKeysPrefix)
writeKeyError(w, err)
reportRequestFailed(rr, err)
return
}
switch {
@ -152,6 +156,7 @@ func (h *keysHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
// Should never be reached
plog.Errorf("error writing event (%v)", err)
}
reportRequestCompleted(rr, resp, startTime)
case resp.Watcher != nil:
ctx, cancel := context.WithTimeout(context.Background(), defaultWatchTimeout)
defer cancel()

View File

@ -0,0 +1,96 @@
// Copyright 2015 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 etcdhttp
import (
"strconv"
"time"
"net/http"
"github.com/coreos/etcd/Godeps/_workspace/src/github.com/prometheus/client_golang/prometheus"
etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/etcd/etcdserver"
"github.com/coreos/etcd/etcdserver/etcdhttp/httptypes"
"github.com/coreos/etcd/etcdserver/etcdserverpb"
)
var (
incomingEvents = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "etcd",
Subsystem: "http",
Name: "received_total",
Help: "Counter of requests received into the system (successfully parsed and authd).",
}, []string{"method"})
failedEvents = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "etcd",
Subsystem: "http",
Name: "failed_total",
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(
prometheus.HistogramOpts{
Namespace: "etcd",
Subsystem: "http",
Name: "successful_duration_second",
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),
}, []string{"method"})
)
func init() {
prometheus.MustRegister(incomingEvents)
prometheus.MustRegister(failedEvents)
prometheus.MustRegister(successfulEventsHandlingTime)
}
func reportRequestReceived(request etcdserverpb.Request) {
incomingEvents.WithLabelValues(methodFromRequest(request)).Inc()
}
func reportRequestCompleted(request etcdserverpb.Request, response etcdserver.Response, startTime time.Time) {
method := methodFromRequest(request)
successfulEventsHandlingTime.WithLabelValues(method).Observe(time.Since(startTime).Seconds())
}
func reportRequestFailed(request etcdserverpb.Request, err error) {
method := methodFromRequest(request)
failedEvents.WithLabelValues(method, strconv.Itoa(codeFromError(err))).Inc()
}
func methodFromRequest(request etcdserverpb.Request) string {
if request.Method == "GET" && request.Quorum {
return "QGET"
}
return request.Method
}
func codeFromError(err error) int {
if err == nil {
return http.StatusInternalServerError
}
switch e := err.(type) {
case *etcdErr.Error:
return (*etcdErr.Error)(e).StatusCode()
case *httptypes.HTTPError:
return (*httptypes.HTTPError)(e).Code
default:
return http.StatusInternalServerError
}
}