Skip to content

Commit

Permalink
*: move v2http handlers not needing v2 store to etcdhttp
Browse files Browse the repository at this point in the history
Lets --enable-v2=false configurations provide /metrics, /health, etc.

Fixes etcd-io#8167
  • Loading branch information
heyitsanthony committed Jul 8, 2017
1 parent 97f37e4 commit 5bbbe8c
Show file tree
Hide file tree
Showing 10 changed files with 308 additions and 192 deletions.
18 changes: 11 additions & 7 deletions embed/etcd.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"time"

"github.com/coreos/etcd/etcdserver"
"github.com/coreos/etcd/etcdserver/api/etcdhttp"
"github.com/coreos/etcd/etcdserver/api/v2http"
"github.com/coreos/etcd/pkg/cors"
"github.com/coreos/etcd/pkg/debugutil"
Expand Down Expand Up @@ -151,7 +152,7 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) {
}

// configure peer handlers after rafthttp.Transport started
ph := v2http.NewPeerHandler(e.Server)
ph := etcdhttp.NewPeerHandler(e.Server)
for i := range e.Peers {
srv := &http.Server{
Handler: ph,
Expand Down Expand Up @@ -384,16 +385,19 @@ func (e *Etcd) serve() (err error) {
}

// Start a client server goroutine for each listen address
var v2h http.Handler
var h http.Handler
if e.Config().EnableV2 {
v2h = http.Handler(&cors.CORSHandler{
Handler: v2http.NewClientHandler(e.Server, e.Server.Cfg.ReqTimeout()),
Info: e.cfg.CorsInfo,
})
h = v2http.NewClientHandler(e.Server, e.Server.Cfg.ReqTimeout())
} else {
mux := http.NewServeMux()
etcdhttp.HandleBasic(mux, e.Server)
h = mux
}
h = http.Handler(&cors.CORSHandler{Handler: h, Info: e.cfg.CorsInfo})

for _, sctx := range e.sctxs {
go func(s *serveCtx) {
e.errHandler(s.serve(e.Server, &e.cfg.ClientTLSInfo, v2h, e.errHandler))
e.errHandler(s.serve(e.Server, &e.cfg.ClientTLSInfo, h, e.errHandler))
}(sctx)
}
return nil
Expand Down
5 changes: 3 additions & 2 deletions error/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,9 +154,10 @@ func (e Error) StatusCode() int {
return status
}

func (e Error) WriteTo(w http.ResponseWriter) {
func (e Error) WriteTo(w http.ResponseWriter) error {
w.Header().Add("X-Etcd-Index", fmt.Sprint(e.Index))
w.Header().Set("Content-Type", "application/json")
w.WriteHeader(e.StatusCode())
fmt.Fprintln(w, e.toJsonString())
_, err := w.Write([]byte(e.toJsonString() + "\n"))
return err
}
186 changes: 186 additions & 0 deletions etcdserver/api/etcdhttp/base.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,186 @@
// Copyright 2015 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 (
"encoding/json"
"expvar"
"fmt"
"net/http"
"strings"
"time"

etcdErr "github.com/coreos/etcd/error"
"github.com/coreos/etcd/etcdserver"
"github.com/coreos/etcd/etcdserver/api"
"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
"github.com/coreos/etcd/etcdserver/etcdserverpb"
"github.com/coreos/etcd/pkg/logutil"
"github.com/coreos/etcd/raft"
"github.com/coreos/etcd/version"
"github.com/coreos/pkg/capnslog"
"github.com/prometheus/client_golang/prometheus"
"golang.org/x/net/context"
)

var (
plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "etcdserver/api/etcdhttp")
mlog = logutil.NewMergeLogger(plog)
)

const (
configPath = "/config"
metricsPath = "/metrics"
healthPath = "/health"
varsPath = "/debug/vars"
versionPath = "/version"
)

// HandleBasic adds handlers to a mux for serving JSON etcd client requests
// that do not access the v2 store.
func HandleBasic(mux *http.ServeMux, server *etcdserver.EtcdServer) {
mux.HandleFunc(varsPath, serveVars)
mux.HandleFunc(configPath+"/local/log", logHandleFunc)
mux.Handle(metricsPath, prometheus.Handler())
mux.Handle(healthPath, healthHandler(server))
mux.HandleFunc(versionPath, versionHandler(server.Cluster(), serveVersion))
}

func healthHandler(server *etcdserver.EtcdServer) http.HandlerFunc {
return func(w http.ResponseWriter, r *http.Request) {
if !allowMethod(w, r, "GET") {
return
}
if uint64(server.Leader()) == raft.None {
http.Error(w, `{"health": "false"}`, http.StatusServiceUnavailable)
return
}
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
defer cancel()
if _, err := server.Do(ctx, etcdserverpb.Request{Method: "QGET"}); err != nil {
http.Error(w, `{"health": "false"}`, http.StatusServiceUnavailable)
return
}
w.WriteHeader(http.StatusOK)
w.Write([]byte(`{"health": "true"}`))
}
}

func versionHandler(c api.Cluster, fn func(http.ResponseWriter, *http.Request, string)) http.HandlerFunc {
return func(w http.ResponseWriter, r *http.Request) {
v := c.Version()
if v != nil {
fn(w, r, v.String())
} else {
fn(w, r, "not_decided")
}
}
}

func serveVersion(w http.ResponseWriter, r *http.Request, clusterV string) {
if !allowMethod(w, r, "GET") {
return
}
vs := version.Versions{
Server: version.Version,
Cluster: clusterV,
}

w.Header().Set("Content-Type", "application/json")
b, err := json.Marshal(&vs)
if err != nil {
plog.Panicf("cannot marshal versions to json (%v)", err)
}
w.Write(b)
}

func logHandleFunc(w http.ResponseWriter, r *http.Request) {
if !allowMethod(w, r, "PUT") {
return
}

in := struct{ Level string }{}

d := json.NewDecoder(r.Body)
if err := d.Decode(&in); err != nil {
WriteError(w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Invalid json body"))
return
}

logl, err := capnslog.ParseLevel(strings.ToUpper(in.Level))
if err != nil {
WriteError(w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Invalid log level "+in.Level))
return
}

plog.Noticef("globalLogLevel set to %q", logl.String())
capnslog.SetGlobalLogLevel(logl)
w.WriteHeader(http.StatusNoContent)
}

func serveVars(w http.ResponseWriter, r *http.Request) {
if !allowMethod(w, r, "GET") {
return
}

w.Header().Set("Content-Type", "application/json; charset=utf-8")
fmt.Fprintf(w, "{\n")
first := true
expvar.Do(func(kv expvar.KeyValue) {
if !first {
fmt.Fprintf(w, ",\n")
}
first = false
fmt.Fprintf(w, "%q: %s", kv.Key, kv.Value)
})
fmt.Fprintf(w, "\n}\n")
}

func allowMethod(w http.ResponseWriter, r *http.Request, m string) bool {
if m == r.Method {
return true
}
w.Header().Set("Allow", m)
http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
return false
}

// WriteError logs and writes the given Error to the ResponseWriter
// If Error is an etcdErr, it is rendered to the ResponseWriter
// Otherwise, it is assumed to be a StatusInternalServerError
func WriteError(w http.ResponseWriter, r *http.Request, err error) {
if err == nil {
return
}
switch e := err.(type) {
case *etcdErr.Error:
e.WriteTo(w)
case *httptypes.HTTPError:
if et := e.WriteTo(w); et != nil {
plog.Debugf("error writing HTTPError (%v) to %s", et, r.RemoteAddr)
}
default:
switch err {
case etcdserver.ErrTimeoutDueToLeaderFail, etcdserver.ErrTimeoutDueToConnectionLost, etcdserver.ErrNotEnoughStartedMembers, etcdserver.ErrUnhealthy:
mlog.MergeError(err)
default:
mlog.MergeErrorf("got unexpected response error (%v)", err)
}
herr := httptypes.NewHTTPError(http.StatusInternalServerError, "Internal Server Error")
if et := herr.WriteTo(w); et != nil {
plog.Debugf("error writing HTTPError (%v) to %s", et, r.RemoteAddr)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package v2http
package etcdhttp

import (
"encoding/json"
Expand Down Expand Up @@ -61,7 +61,7 @@ type peerMembersHandler struct {
}

func (h *peerMembersHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
if !allowMethod(w, r.Method, "GET") {
if !allowMethod(w, r, "GET") {
return
}
w.Header().Set("X-Etcd-Cluster-ID", h.cluster.ID().String())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,21 +12,44 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package v2http
package etcdhttp

import (
"encoding/json"
"io/ioutil"
"net/http"
"net/http/httptest"
"path"
"sort"
"testing"

"github.com/coreos/etcd/etcdserver/membership"
"github.com/coreos/etcd/pkg/testutil"
"github.com/coreos/etcd/pkg/types"
"github.com/coreos/etcd/rafthttp"
"github.com/coreos/go-semver/semver"
)

type fakeCluster struct {
id uint64
clientURLs []string
members map[uint64]*membership.Member
}

func (c *fakeCluster) ID() types.ID { return types.ID(c.id) }
func (c *fakeCluster) ClientURLs() []string { return c.clientURLs }
func (c *fakeCluster) Members() []*membership.Member {
var ms membership.MembersByID
for _, m := range c.members {
ms = append(ms, m)
}
sort.Sort(ms)
return []*membership.Member(ms)
}
func (c *fakeCluster) Member(id types.ID) *membership.Member { return c.members[uint64(id)] }
func (c *fakeCluster) IsIDRemoved(id types.ID) bool { return false }
func (c *fakeCluster) Version() *semver.Version { return nil }

// TestNewPeerHandlerOnRaftPrefix tests that NewPeerHandler returns a handler that
// handles raft-prefix requests well.
func TestNewPeerHandlerOnRaftPrefix(t *testing.T) {
Expand Down
66 changes: 66 additions & 0 deletions etcdserver/api/etcdhttp/version_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
// 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 (
"encoding/json"
"net/http"
"net/http/httptest"
"testing"

"github.com/coreos/etcd/version"
)

func TestServeVersion(t *testing.T) {
req, err := http.NewRequest("GET", "", nil)
if err != nil {
t.Fatalf("error creating request: %v", err)
}
rw := httptest.NewRecorder()
serveVersion(rw, req, "2.1.0")
if rw.Code != http.StatusOK {
t.Errorf("code=%d, want %d", rw.Code, http.StatusOK)
}
vs := version.Versions{
Server: version.Version,
Cluster: "2.1.0",
}
w, err := json.Marshal(&vs)
if err != nil {
t.Fatal(err)
}
if g := rw.Body.String(); g != string(w) {
t.Fatalf("body = %q, want %q", g, string(w))
}
if ct := rw.HeaderMap.Get("Content-Type"); ct != "application/json" {
t.Errorf("contet-type header = %s, want %s", ct, "application/json")
}
}

func TestServeVersionFails(t *testing.T) {
for _, m := range []string{
"CONNECT", "TRACE", "PUT", "POST", "HEAD",
} {
req, err := http.NewRequest(m, "", nil)
if err != nil {
t.Fatalf("error creating request: %v", err)
}
rw := httptest.NewRecorder()
serveVersion(rw, req, "2.1.0")
if rw.Code != http.StatusMethodNotAllowed {
t.Errorf("method %s: code=%d, want %d", m, rw.Code, http.StatusMethodNotAllowed)
}
}
}
Loading

0 comments on commit 5bbbe8c

Please sign in to comment.