diff --git a/server/embed/etcd.go b/server/embed/etcd.go index 67764968f..c26f4f30d 100644 --- a/server/embed/etcd.go +++ b/server/embed/etcd.go @@ -746,7 +746,8 @@ func (e *Etcd) serveClients() (err error) { } else { mux := http.NewServeMux() etcdhttp.HandleBasic(e.cfg.logger, mux, e.Server) - etcdhttp.HandleMetricsHealthForV3(e.cfg.logger, mux, e.Server) + etcdhttp.HandleMetrics(mux) + etcdhttp.HandleHealth(e.cfg.logger, mux, e.Server) h = mux } @@ -835,7 +836,8 @@ func (e *Etcd) serveMetrics() (err error) { if len(e.cfg.ListenMetricsUrls) > 0 { metricsMux := http.NewServeMux() - etcdhttp.HandleMetricsHealthForV3(e.cfg.logger, metricsMux, e.Server) + etcdhttp.HandleMetrics(metricsMux) + etcdhttp.HandleHealth(e.cfg.logger, metricsMux, e.Server) for _, murl := range e.cfg.ListenMetricsUrls { tlsInfo := &e.cfg.ClientTLSInfo diff --git a/server/etcdmain/etcd.go b/server/etcdmain/etcd.go index 6a79f0a78..9921b89df 100644 --- a/server/etcdmain/etcd.go +++ b/server/etcdmain/etcd.go @@ -416,7 +416,7 @@ func startProxy(cfg *config) error { go func() { lg.Info("v2 proxy started listening on client requests", zap.String("host", host)) mux := http.NewServeMux() - etcdhttp.HandlePrometheus(mux) // v2 proxy just uses the same port + etcdhttp.HandleMetrics(mux) // v2 proxy just uses the same port mux.Handle("/", ph) lg.Fatal("done serving", zap.Error(http.Serve(l, mux))) }() diff --git a/server/etcdserver/api/etcdhttp/health.go b/server/etcdserver/api/etcdhttp/health.go new file mode 100644 index 000000000..3d9524266 --- /dev/null +++ b/server/etcdserver/api/etcdhttp/health.go @@ -0,0 +1,229 @@ +// Copyright 2017 The etcd Authors +// +// 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 ( + "context" + "encoding/json" + "fmt" + "net/http" + "time" + + "go.uber.org/zap" + + "github.com/prometheus/client_golang/prometheus" + pb "go.etcd.io/etcd/api/v3/etcdserverpb" + "go.etcd.io/etcd/client/pkg/v3/types" + "go.etcd.io/etcd/raft/v3" + "go.etcd.io/etcd/server/v3/auth" + "go.etcd.io/etcd/server/v3/config" + "go.etcd.io/etcd/server/v3/etcdserver" +) + +const ( + PathHealth = "/health" + PathProxyHealth = "/proxy/health" +) + +type ServerHealth interface { + serverHealthV2V3 + Range(context.Context, *pb.RangeRequest) (*pb.RangeResponse, error) + Config() config.ServerConfig +} + +type serverHealthV2V3 interface { + Alarms() []*pb.AlarmMember + Leader() types.ID +} + +// HandleHealth registers metrics and health handlers for v2. +func HandleHealthForV2(lg *zap.Logger, mux *http.ServeMux, srv etcdserver.ServerV2) { + mux.Handle(PathHealth, NewHealthHandler(lg, func(excludedAlarms AlarmSet, serializable bool) Health { + if h := checkAlarms(lg, srv, excludedAlarms); h.Health != "true" { + return h + } + if h := checkLeader(lg, srv, serializable); h.Health != "true" { + return h + } + return checkV2API(lg, srv) + })) +} + +// HandleHealth registers metrics and health handlers. it checks health by using v3 range request +// and its corresponding timeout. +func HandleHealth(lg *zap.Logger, mux *http.ServeMux, srv ServerHealth) { + mux.Handle(PathHealth, NewHealthHandler(lg, func(excludedAlarms AlarmSet, serializable bool) Health { + if h := checkAlarms(lg, srv, excludedAlarms); h.Health != "true" { + return h + } + if h := checkLeader(lg, srv, serializable); h.Health != "true" { + return h + } + return checkAPI(lg, srv, serializable) + })) +} + +// NewHealthHandler handles '/health' requests. +func NewHealthHandler(lg *zap.Logger, hfunc func(excludedAlarms AlarmSet, Serializable bool) Health) http.HandlerFunc { + return func(w http.ResponseWriter, r *http.Request) { + if r.Method != http.MethodGet { + w.Header().Set("Allow", http.MethodGet) + http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed) + lg.Warn("/health error", zap.Int("status-code", http.StatusMethodNotAllowed)) + return + } + excludedAlarms := getExcludedAlarms(r) + // Passing the query parameter "serializable=true" ensures that the + // health of the local etcd is checked vs the health of the cluster. + // This is useful for probes attempting to validate the liveness of + // the etcd process vs readiness of the cluster to serve requests. + serializableFlag := getSerializableFlag(r) + h := hfunc(excludedAlarms, serializableFlag) + defer func() { + if h.Health == "true" { + healthSuccess.Inc() + } else { + healthFailed.Inc() + } + }() + d, _ := json.Marshal(h) + if h.Health != "true" { + http.Error(w, string(d), http.StatusServiceUnavailable) + lg.Warn("/health error", zap.String("output", string(d)), zap.Int("status-code", http.StatusServiceUnavailable)) + return + } + w.WriteHeader(http.StatusOK) + w.Write(d) + lg.Debug("/health OK", zap.Int("status-code", http.StatusOK)) + } +} + +var ( + healthSuccess = prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: "etcd", + Subsystem: "server", + Name: "health_success", + Help: "The total number of successful health checks", + }) + healthFailed = prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: "etcd", + Subsystem: "server", + Name: "health_failures", + Help: "The total number of failed health checks", + }) +) + +func init() { + prometheus.MustRegister(healthSuccess) + prometheus.MustRegister(healthFailed) +} + +// Health defines etcd server health status. +// TODO: remove manual parsing in etcdctl cluster-health +type Health struct { + Health string `json:"health"` + Reason string `json:"reason"` +} + +type AlarmSet map[string]struct{} + +func getExcludedAlarms(r *http.Request) (alarms AlarmSet) { + alarms = make(map[string]struct{}, 2) + alms, found := r.URL.Query()["exclude"] + if found { + for _, alm := range alms { + if len(alm) == 0 { + continue + } + alarms[alm] = struct{}{} + } + } + return alarms +} + +func getSerializableFlag(r *http.Request) bool { + return r.URL.Query().Get("serializable") == "true" +} + +// TODO: etcdserver.ErrNoLeader in health API + +func checkAlarms(lg *zap.Logger, srv serverHealthV2V3, excludedAlarms AlarmSet) Health { + h := Health{Health: "true"} + as := srv.Alarms() + if len(as) > 0 { + for _, v := range as { + alarmName := v.Alarm.String() + if _, found := excludedAlarms[alarmName]; found { + lg.Debug("/health excluded alarm", zap.String("alarm", v.String())) + continue + } + + h.Health = "false" + switch v.Alarm { + case pb.AlarmType_NOSPACE: + h.Reason = "ALARM NOSPACE" + case pb.AlarmType_CORRUPT: + h.Reason = "ALARM CORRUPT" + default: + h.Reason = "ALARM UNKNOWN" + } + lg.Warn("serving /health false due to an alarm", zap.String("alarm", v.String())) + return h + } + } + + return h +} + +func checkLeader(lg *zap.Logger, srv serverHealthV2V3, serializable bool) Health { + h := Health{Health: "true"} + if !serializable && (uint64(srv.Leader()) == raft.None) { + h.Health = "false" + h.Reason = "RAFT NO LEADER" + lg.Warn("serving /health false; no leader") + } + return h +} + +func checkV2API(lg *zap.Logger, srv etcdserver.ServerV2) Health { + h := Health{Health: "true"} + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + _, err := srv.Do(ctx, pb.Request{Method: "QGET"}) + cancel() + if err != nil { + h.Health = "false" + h.Reason = fmt.Sprintf("QGET ERROR:%s", err) + lg.Warn("serving /health false; QGET fails", zap.Error(err)) + return h + } + lg.Debug("serving /health true") + return h +} + +func checkAPI(lg *zap.Logger, srv ServerHealth, serializable bool) Health { + h := Health{Health: "true"} + cfg := srv.Config() + ctx, cancel := context.WithTimeout(context.Background(), cfg.ReqTimeout()) + _, err := srv.Range(ctx, &pb.RangeRequest{KeysOnly: true, Limit: 1, Serializable: serializable}) + cancel() + if err != nil && err != auth.ErrUserEmpty && err != auth.ErrPermissionDenied { + h.Health = "false" + h.Reason = fmt.Sprintf("RANGE ERROR:%s", err) + lg.Warn("serving /health false; Range fails", zap.Error(err)) + return h + } + lg.Debug("serving /health true") + return h +} diff --git a/server/etcdserver/api/etcdhttp/health_test.go b/server/etcdserver/api/etcdhttp/health_test.go new file mode 100644 index 000000000..27ad30be4 --- /dev/null +++ b/server/etcdserver/api/etcdhttp/health_test.go @@ -0,0 +1,177 @@ +package etcdhttp + +import ( + "context" + "encoding/json" + "fmt" + "io" + "net/http" + "net/http/httptest" + "testing" + + pb "go.etcd.io/etcd/api/v3/etcdserverpb" + "go.etcd.io/etcd/client/pkg/v3/testutil" + "go.etcd.io/etcd/client/pkg/v3/types" + "go.etcd.io/etcd/raft/v3" + "go.etcd.io/etcd/server/v3/auth" + "go.etcd.io/etcd/server/v3/config" + "go.etcd.io/etcd/server/v3/etcdserver" + "go.uber.org/zap/zaptest" +) + +type fakeHealthServer struct { + fakeServer + health string + apiError error +} + +func (s *fakeHealthServer) Range(ctx context.Context, request *pb.RangeRequest) (*pb.RangeResponse, error) { + return nil, s.apiError +} + +func (s *fakeHealthServer) Config() config.ServerConfig { + return config.ServerConfig{} +} + +func (s *fakeHealthServer) Leader() types.ID { + if s.health == "true" { + return 1 + } + return types.ID(raft.None) +} +func (s *fakeHealthServer) Do(ctx context.Context, r pb.Request) (etcdserver.Response, error) { + if s.health == "true" { + return etcdserver.Response{}, nil + } + return etcdserver.Response{}, fmt.Errorf("fail health check") +} +func (s *fakeHealthServer) ClientCertAuthEnabled() bool { return false } + +func TestHealthHandler(t *testing.T) { + // define the input and expected output + // input: alarms, and healthCheckURL + tests := []struct { + name string + alarms []*pb.AlarmMember + healthCheckURL string + apiError error + + expectStatusCode int + expectHealth string + }{ + { + name: "Healthy if no alarm", + alarms: []*pb.AlarmMember{}, + healthCheckURL: "/health", + expectStatusCode: http.StatusOK, + expectHealth: "true", + }, + { + name: "Unhealthy if NOSPACE alarm is on", + alarms: []*pb.AlarmMember{{MemberID: uint64(0), Alarm: pb.AlarmType_NOSPACE}}, + healthCheckURL: "/health", + expectStatusCode: http.StatusServiceUnavailable, + expectHealth: "false", + }, + { + name: "Healthy if NOSPACE alarm is on and excluded", + alarms: []*pb.AlarmMember{{MemberID: uint64(0), Alarm: pb.AlarmType_NOSPACE}}, + healthCheckURL: "/health?exclude=NOSPACE", + expectStatusCode: http.StatusOK, + expectHealth: "true", + }, + { + name: "Healthy if NOSPACE alarm is excluded", + alarms: []*pb.AlarmMember{}, + healthCheckURL: "/health?exclude=NOSPACE", + expectStatusCode: http.StatusOK, + expectHealth: "true", + }, + { + name: "Healthy if multiple NOSPACE alarms are on and excluded", + alarms: []*pb.AlarmMember{{MemberID: uint64(1), Alarm: pb.AlarmType_NOSPACE}, {MemberID: uint64(2), Alarm: pb.AlarmType_NOSPACE}, {MemberID: uint64(3), Alarm: pb.AlarmType_NOSPACE}}, + healthCheckURL: "/health?exclude=NOSPACE", + expectStatusCode: http.StatusOK, + expectHealth: "true", + }, + { + name: "Unhealthy if NOSPACE alarms is excluded and CORRUPT is on", + alarms: []*pb.AlarmMember{{MemberID: uint64(0), Alarm: pb.AlarmType_NOSPACE}, {MemberID: uint64(1), Alarm: pb.AlarmType_CORRUPT}}, + healthCheckURL: "/health?exclude=NOSPACE", + expectStatusCode: http.StatusServiceUnavailable, + expectHealth: "false", + }, + { + name: "Unhealthy if both NOSPACE and CORRUPT are on and excluded", + alarms: []*pb.AlarmMember{{MemberID: uint64(0), Alarm: pb.AlarmType_NOSPACE}, {MemberID: uint64(1), Alarm: pb.AlarmType_CORRUPT}}, + healthCheckURL: "/health?exclude=NOSPACE&exclude=CORRUPT", + expectStatusCode: http.StatusOK, + expectHealth: "true", + }, + { + name: "Healthy even if authentication failed", + healthCheckURL: "/health", + apiError: auth.ErrUserEmpty, + expectStatusCode: http.StatusOK, + expectHealth: "true", + }, + { + name: "Healthy even if authorization failed", + healthCheckURL: "/health", + apiError: auth.ErrPermissionDenied, + expectStatusCode: http.StatusOK, + expectHealth: "true", + }, + { + name: "Unhealthy if api is not available", + healthCheckURL: "/health", + apiError: fmt.Errorf("Unexpected error"), + expectStatusCode: http.StatusServiceUnavailable, + expectHealth: "false", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + mux := http.NewServeMux() + HandleHealth(zaptest.NewLogger(t), mux, &fakeHealthServer{ + fakeServer: fakeServer{alarms: tt.alarms}, + health: tt.expectHealth, + apiError: tt.apiError, + }) + ts := httptest.NewServer(mux) + defer ts.Close() + + res, err := ts.Client().Do(&http.Request{Method: http.MethodGet, URL: testutil.MustNewURL(t, ts.URL+tt.healthCheckURL)}) + if err != nil { + t.Errorf("fail serve http request %s %v", tt.healthCheckURL, err) + } + if res == nil { + t.Errorf("got nil http response with http request %s", tt.healthCheckURL) + return + } + if res.StatusCode != tt.expectStatusCode { + t.Errorf("want statusCode %d but got %d", tt.expectStatusCode, res.StatusCode) + } + health, err := parseHealthOutput(res.Body) + if err != nil { + t.Errorf("fail parse health check output %v", err) + } + if health.Health != tt.expectHealth { + t.Errorf("want health %s but got %s", tt.expectHealth, health.Health) + } + }) + } +} + +func parseHealthOutput(body io.Reader) (Health, error) { + obj := Health{} + d, derr := io.ReadAll(body) + if derr != nil { + return obj, derr + } + if err := json.Unmarshal(d, &obj); err != nil { + return obj, err + } + return obj, nil +} diff --git a/server/etcdserver/api/etcdhttp/metrics.go b/server/etcdserver/api/etcdhttp/metrics.go index e16955fb1..bf7d4a4a4 100644 --- a/server/etcdserver/api/etcdhttp/metrics.go +++ b/server/etcdserver/api/etcdhttp/metrics.go @@ -15,197 +15,17 @@ package etcdhttp import ( - "context" - "encoding/json" - "fmt" "net/http" - "time" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" - "go.etcd.io/etcd/api/v3/etcdserverpb" - "go.etcd.io/etcd/raft/v3" - "go.etcd.io/etcd/server/v3/auth" - "go.etcd.io/etcd/server/v3/etcdserver" - "go.uber.org/zap" ) const ( PathMetrics = "/metrics" - PathHealth = "/health" PathProxyMetrics = "/proxy/metrics" - PathProxyHealth = "/proxy/health" ) -// HandleMetricsHealth registers metrics and health handlers. -func HandleMetricsHealth(lg *zap.Logger, mux *http.ServeMux, srv etcdserver.ServerV2) { - mux.Handle(PathMetrics, promhttp.Handler()) - mux.Handle(PathHealth, NewHealthHandler(lg, func(excludedAlarms AlarmSet, serializable bool) Health { return checkV2Health(lg, srv, excludedAlarms) })) -} - -// HandleMetricsHealthForV3 registers metrics and health handlers. it checks health by using v3 range request -// and its corresponding timeout. -func HandleMetricsHealthForV3(lg *zap.Logger, mux *http.ServeMux, srv *etcdserver.EtcdServer) { - mux.Handle(PathMetrics, promhttp.Handler()) - mux.Handle(PathHealth, NewHealthHandler(lg, func(excludedAlarms AlarmSet, serializable bool) Health { - return checkV3Health(lg, srv, excludedAlarms, serializable) - })) -} - -// HandlePrometheus registers prometheus handler on '/metrics'. -func HandlePrometheus(mux *http.ServeMux) { +// HandleMetrics registers prometheus handler on '/metrics'. +func HandleMetrics(mux *http.ServeMux) { mux.Handle(PathMetrics, promhttp.Handler()) } - -// NewHealthHandler handles '/health' requests. -func NewHealthHandler(lg *zap.Logger, hfunc func(excludedAlarms AlarmSet, serializable bool) Health) http.HandlerFunc { - return func(w http.ResponseWriter, r *http.Request) { - if r.Method != http.MethodGet { - w.Header().Set("Allow", http.MethodGet) - http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed) - lg.Warn("/health error", zap.Int("status-code", http.StatusMethodNotAllowed)) - return - } - excludedAlarms := getExcludedAlarms(r) - // Passing the query parameter "serializable=true" ensures that the - // health of the local etcd is checked vs the health of the cluster. - // This is useful for probes attempting to validate the liveness of - // the etcd process vs readiness of the cluster to serve requests. - serializableFlag := getSerializableFlag(r) - h := hfunc(excludedAlarms, serializableFlag) - defer func() { - if h.Health == "true" { - healthSuccess.Inc() - } else { - healthFailed.Inc() - } - }() - d, _ := json.Marshal(h) - if h.Health != "true" { - http.Error(w, string(d), http.StatusServiceUnavailable) - lg.Warn("/health error", zap.String("output", string(d)), zap.Int("status-code", http.StatusServiceUnavailable)) - return - } - w.WriteHeader(http.StatusOK) - w.Write(d) - lg.Debug("/health OK", zap.Int("status-code", http.StatusOK)) - } -} - -var ( - healthSuccess = prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: "etcd", - Subsystem: "server", - Name: "health_success", - Help: "The total number of successful health checks", - }) - healthFailed = prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: "etcd", - Subsystem: "server", - Name: "health_failures", - Help: "The total number of failed health checks", - }) -) - -func init() { - prometheus.MustRegister(healthSuccess) - prometheus.MustRegister(healthFailed) -} - -// Health defines etcd server health status. -// TODO: remove manual parsing in etcdctl cluster-health -type Health struct { - Health string `json:"health"` - Reason string `json:"reason"` -} - -type AlarmSet map[string]struct{} - -func getExcludedAlarms(r *http.Request) (alarms AlarmSet) { - alarms = make(map[string]struct{}, 2) - alms, found := r.URL.Query()["exclude"] - if found { - for _, alm := range alms { - if len(alms) == 0 { - continue - } - alarms[alm] = struct{}{} - } - } - return alarms -} - -func getSerializableFlag(r *http.Request) bool { - return r.URL.Query().Get("serializable") == "true" -} - -// TODO: etcdserver.ErrNoLeader in health API - -func checkHealth(lg *zap.Logger, srv etcdserver.ServerV2, excludedAlarms AlarmSet, serializable bool) Health { - h := Health{} - h.Health = "true" - as := srv.Alarms() - if len(as) > 0 { - for _, v := range as { - alarmName := v.Alarm.String() - if _, found := excludedAlarms[alarmName]; found { - lg.Debug("/health excluded alarm", zap.String("alarm", v.String())) - continue - } - - h.Health = "false" - switch v.Alarm { - case etcdserverpb.AlarmType_NOSPACE: - h.Reason = "ALARM NOSPACE" - case etcdserverpb.AlarmType_CORRUPT: - h.Reason = "ALARM CORRUPT" - default: - h.Reason = "ALARM UNKNOWN" - } - lg.Warn("serving /health false due to an alarm", zap.String("alarm", v.String())) - return h - } - } - - if !serializable && uint64(srv.Leader()) == raft.None { - h.Health = "false" - h.Reason = "RAFT NO LEADER" - lg.Warn("serving /health false; no leader") - return h - } - return h -} - -func checkV2Health(lg *zap.Logger, srv etcdserver.ServerV2, excludedAlarms AlarmSet) (h Health) { - if h = checkHealth(lg, srv, excludedAlarms, false); h.Health != "true" { - return - } - ctx, cancel := context.WithTimeout(context.Background(), time.Second) - _, err := srv.Do(ctx, etcdserverpb.Request{Method: "QGET"}) - cancel() - if err != nil { - h.Health = "false" - h.Reason = fmt.Sprintf("QGET ERROR:%s", err) - lg.Warn("serving /health false; QGET fails", zap.Error(err)) - return - } - lg.Debug("serving /health true") - return -} - -func checkV3Health(lg *zap.Logger, srv *etcdserver.EtcdServer, excludedAlarms AlarmSet, serializable bool) (h Health) { - if h = checkHealth(lg, srv, excludedAlarms, serializable); h.Health != "true" { - return - } - ctx, cancel := context.WithTimeout(context.Background(), srv.Cfg.ReqTimeout()) - _, err := srv.Range(ctx, &etcdserverpb.RangeRequest{KeysOnly: true, Limit: 1, Serializable: serializable}) - cancel() - if err != nil && err != auth.ErrUserEmpty && err != auth.ErrPermissionDenied { - h.Health = "false" - h.Reason = fmt.Sprintf("RANGE ERROR:%s", err) - lg.Warn("serving /health false; Range fails", zap.Error(err)) - return - } - lg.Debug("serving /health true") - return -} diff --git a/server/etcdserver/api/etcdhttp/metrics_test.go b/server/etcdserver/api/etcdhttp/metrics_test.go deleted file mode 100644 index d07d8c46f..000000000 --- a/server/etcdserver/api/etcdhttp/metrics_test.go +++ /dev/null @@ -1,144 +0,0 @@ -package etcdhttp - -import ( - "context" - "encoding/json" - "fmt" - "io" - "io/ioutil" - "net/http" - "net/http/httptest" - "testing" - - pb "go.etcd.io/etcd/api/v3/etcdserverpb" - "go.etcd.io/etcd/client/pkg/v3/testutil" - "go.etcd.io/etcd/client/pkg/v3/types" - "go.etcd.io/etcd/raft/v3" - "go.etcd.io/etcd/server/v3/etcdserver" - stats "go.etcd.io/etcd/server/v3/etcdserver/api/v2stats" - "go.uber.org/zap" -) - -type fakeStats struct{} - -func (s *fakeStats) SelfStats() []byte { return nil } -func (s *fakeStats) LeaderStats() []byte { return nil } -func (s *fakeStats) StoreStats() []byte { return nil } - -type fakeServerV2 struct { - fakeServer - stats.Stats - health string -} - -func (s *fakeServerV2) Leader() types.ID { - if s.health == "true" { - return 1 - } - return types.ID(raft.None) -} -func (s *fakeServerV2) Do(ctx context.Context, r pb.Request) (etcdserver.Response, error) { - if s.health == "true" { - return etcdserver.Response{}, nil - } - return etcdserver.Response{}, fmt.Errorf("fail health check") -} -func (s *fakeServerV2) ClientCertAuthEnabled() bool { return false } - -func TestHealthHandler(t *testing.T) { - // define the input and expected output - // input: alarms, and healthCheckURL - tests := []struct { - alarms []*pb.AlarmMember - healthCheckURL string - statusCode int - health string - }{ - { - []*pb.AlarmMember{}, - "/health", - http.StatusOK, - "true", - }, - { - []*pb.AlarmMember{{MemberID: uint64(0), Alarm: pb.AlarmType_NOSPACE}}, - "/health", - http.StatusServiceUnavailable, - "false", - }, - { - []*pb.AlarmMember{{MemberID: uint64(0), Alarm: pb.AlarmType_NOSPACE}}, - "/health?exclude=NOSPACE", - http.StatusOK, - "true", - }, - { - []*pb.AlarmMember{}, - "/health?exclude=NOSPACE", - http.StatusOK, - "true", - }, - { - []*pb.AlarmMember{{MemberID: uint64(1), Alarm: pb.AlarmType_NOSPACE}, {MemberID: uint64(2), Alarm: pb.AlarmType_NOSPACE}, {MemberID: uint64(3), Alarm: pb.AlarmType_NOSPACE}}, - "/health?exclude=NOSPACE", - http.StatusOK, - "true", - }, - { - []*pb.AlarmMember{{MemberID: uint64(0), Alarm: pb.AlarmType_NOSPACE}, {MemberID: uint64(1), Alarm: pb.AlarmType_CORRUPT}}, - "/health?exclude=NOSPACE", - http.StatusServiceUnavailable, - "false", - }, - { - []*pb.AlarmMember{{MemberID: uint64(0), Alarm: pb.AlarmType_NOSPACE}, {MemberID: uint64(1), Alarm: pb.AlarmType_CORRUPT}}, - "/health?exclude=NOSPACE&exclude=CORRUPT", - http.StatusOK, - "true", - }, - } - - for i, tt := range tests { - func() { - mux := http.NewServeMux() - HandleMetricsHealth(zap.NewExample(), mux, &fakeServerV2{ - fakeServer: fakeServer{alarms: tt.alarms}, - Stats: &fakeStats{}, - health: tt.health, - }) - ts := httptest.NewServer(mux) - defer ts.Close() - - res, err := ts.Client().Do(&http.Request{Method: http.MethodGet, URL: testutil.MustNewURL(t, ts.URL+tt.healthCheckURL)}) - if err != nil { - t.Errorf("fail serve http request %s %v in test case #%d", tt.healthCheckURL, err, i+1) - } - if res == nil { - t.Errorf("got nil http response with http request %s in test case #%d", tt.healthCheckURL, i+1) - return - } - if res.StatusCode != tt.statusCode { - t.Errorf("want statusCode %d but got %d in test case #%d", tt.statusCode, res.StatusCode, i+1) - } - health, err := parseHealthOutput(res.Body) - if err != nil { - t.Errorf("fail parse health check output %v", err) - } - if health.Health != tt.health { - t.Errorf("want health %s but got %s", tt.health, health.Health) - } - }() - } -} - -func parseHealthOutput(body io.Reader) (Health, error) { - obj := Health{} - d, derr := ioutil.ReadAll(body) - if derr != nil { - return obj, derr - } - if err := json.Unmarshal(d, &obj); err != nil { - return obj, err - } - return obj, nil -} diff --git a/server/etcdserver/api/v2http/client.go b/server/etcdserver/api/v2http/client.go index 17b420732..789945950 100644 --- a/server/etcdserver/api/v2http/client.go +++ b/server/etcdserver/api/v2http/client.go @@ -58,7 +58,8 @@ func NewClientHandler(lg *zap.Logger, server etcdserver.ServerPeer, timeout time } mux := http.NewServeMux() etcdhttp.HandleBasic(lg, mux, server) - etcdhttp.HandleMetricsHealth(lg, mux, server) + etcdhttp.HandleMetrics(mux) + etcdhttp.HandleHealthForV2(lg, mux, server) handleV2(lg, mux, server, timeout) return requestLogger(lg, mux) } diff --git a/server/etcdserver/server.go b/server/etcdserver/server.go index 4e1c2c041..5f62ddc26 100644 --- a/server/etcdserver/server.go +++ b/server/etcdserver/server.go @@ -724,6 +724,10 @@ func (s *EtcdServer) Logger() *zap.Logger { return l } +func (s *EtcdServer) Config() config.ServerConfig { + return s.Cfg +} + func tickToDur(ticks int, tickMs uint) string { return fmt.Sprintf("%v", time.Duration(ticks)*time.Duration(tickMs)*time.Millisecond) }