Browse Source

*: move v2http handlers without /v2 prefix to etcdhttp

Lets --enable-v2=false configurations provide /metrics, /health, etc.

Fixes #8167
Anthony Romano 8 years ago
parent
commit
e29db923bc

+ 11 - 7
embed/etcd.go

@@ -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"
@@ -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,
@@ -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

+ 3 - 2
error/error.go

@@ -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 - 0
etcdserver/api/etcdhttp/base.go

@@ -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)
+		}
+	}
+}

+ 2 - 2
etcdserver/api/v2http/peer.go → etcdserver/api/etcdhttp/peer.go

@@ -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"
@@ -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())

+ 24 - 1
etcdserver/api/v2http/peer_test.go → etcdserver/api/etcdhttp/peer_test.go

@@ -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"
@@ -20,13 +20,36 @@ import (
 	"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) {

+ 66 - 0
etcdserver/api/etcdhttp/version_test.go

@@ -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)
+		}
+	}
+}

+ 9 - 111
etcdserver/api/v2http/client.go

@@ -17,7 +17,6 @@ package v2http
 import (
 	"encoding/json"
 	"errors"
-	"expvar"
 	"fmt"
 	"io/ioutil"
 	"net/http"
@@ -30,18 +29,15 @@ import (
 	etcdErr "github.com/coreos/etcd/error"
 	"github.com/coreos/etcd/etcdserver"
 	"github.com/coreos/etcd/etcdserver/api"
+	"github.com/coreos/etcd/etcdserver/api/etcdhttp"
 	"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
 	"github.com/coreos/etcd/etcdserver/auth"
 	"github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"github.com/coreos/etcd/etcdserver/membership"
 	"github.com/coreos/etcd/etcdserver/stats"
 	"github.com/coreos/etcd/pkg/types"
-	"github.com/coreos/etcd/raft"
 	"github.com/coreos/etcd/store"
-	"github.com/coreos/etcd/version"
-	"github.com/coreos/pkg/capnslog"
 	"github.com/jonboulle/clockwork"
-	"github.com/prometheus/client_golang/prometheus"
 	"golang.org/x/net/context"
 )
 
@@ -51,17 +47,18 @@ const (
 	machinesPrefix = "/v2/machines"
 	membersPrefix  = "/v2/members"
 	statsPrefix    = "/v2/stats"
-	varsPath       = "/debug/vars"
-	metricsPath    = "/metrics"
-	healthPath     = "/health"
-	versionPath    = "/version"
-	configPath     = "/config"
 )
 
 // NewClientHandler generates a muxed http.Handler with the given parameters to serve etcd client requests.
 func NewClientHandler(server *etcdserver.EtcdServer, timeout time.Duration) http.Handler {
-	sec := auth.NewStore(server, timeout)
+	mux := http.NewServeMux()
+	etcdhttp.HandleBasic(mux, server)
+	handleV2(mux, server, timeout)
+	return requestLogger(mux)
+}
 
+func handleV2(mux *http.ServeMux, server *etcdserver.EtcdServer, timeout time.Duration) {
+	sec := auth.NewStore(server, timeout)
 	kh := &keysHandler{
 		sec:                   sec,
 		server:                server,
@@ -91,25 +88,16 @@ func NewClientHandler(server *etcdserver.EtcdServer, timeout time.Duration) http
 		cluster:               server.Cluster(),
 		clientCertAuthEnabled: server.Cfg.ClientCertAuthEnabled,
 	}
-
-	mux := http.NewServeMux()
 	mux.HandleFunc("/", http.NotFound)
-	mux.Handle(healthPath, healthHandler(server))
-	mux.HandleFunc(versionPath, versionHandler(server.Cluster(), serveVersion))
 	mux.Handle(keysPrefix, kh)
 	mux.Handle(keysPrefix+"/", kh)
 	mux.HandleFunc(statsPrefix+"/store", sh.serveStore)
 	mux.HandleFunc(statsPrefix+"/self", sh.serveSelf)
 	mux.HandleFunc(statsPrefix+"/leader", sh.serveLeader)
-	mux.HandleFunc(varsPath, serveVars)
-	mux.HandleFunc(configPath+"/local/log", logHandleFunc)
-	mux.Handle(metricsPath, prometheus.Handler())
 	mux.Handle(membersPrefix, mh)
 	mux.Handle(membersPrefix+"/", mh)
 	mux.Handle(machinesPrefix, mah)
 	handleAuth(mux, sech)
-
-	return requestLogger(mux)
 }
 
 type keysHandler struct {
@@ -319,103 +307,13 @@ func (h *statsHandler) serveLeader(w http.ResponseWriter, r *http.Request) {
 	}
 	stats := h.stats.LeaderStats()
 	if stats == nil {
-		writeError(w, r, httptypes.NewHTTPError(http.StatusForbidden, "not current leader"))
+		etcdhttp.WriteError(w, r, httptypes.NewHTTPError(http.StatusForbidden, "not current leader"))
 		return
 	}
 	w.Header().Set("Content-Type", "application/json")
 	w.Write(stats)
 }
 
-func serveVars(w http.ResponseWriter, r *http.Request) {
-	if !allowMethod(w, r.Method, "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 healthHandler(server *etcdserver.EtcdServer) http.HandlerFunc {
-	return func(w http.ResponseWriter, r *http.Request) {
-		if !allowMethod(w, r.Method, "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.Method, "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.Method, "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)
-}
-
 // parseKeyRequest converts a received http.Request on keysPrefix to
 // a server Request, performing validation of supplied fields as appropriate.
 // If any validation fails, an empty Request and non-nil error is returned.

+ 0 - 43
etcdserver/api/v2http/client_test.go

@@ -37,7 +37,6 @@ import (
 	"github.com/coreos/etcd/pkg/types"
 	"github.com/coreos/etcd/raft/raftpb"
 	"github.com/coreos/etcd/store"
-	"github.com/coreos/etcd/version"
 	"github.com/coreos/go-semver/semver"
 	"github.com/jonboulle/clockwork"
 	"golang.org/x/net/context"
@@ -1409,48 +1408,6 @@ func TestServeStoreStats(t *testing.T) {
 
 }
 
-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)
-		}
-	}
-}
-
 func TestBadServeKeys(t *testing.T) {
 	testBadCases := []struct {
 		req    *http.Request

+ 5 - 25
etcdserver/api/v2http/http.go

@@ -20,12 +20,11 @@ import (
 	"strings"
 	"time"
 
-	etcdErr "github.com/coreos/etcd/error"
-	"github.com/coreos/etcd/etcdserver"
+	"github.com/coreos/etcd/etcdserver/api/etcdhttp"
 	"github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
-
 	"github.com/coreos/etcd/etcdserver/auth"
 	"github.com/coreos/etcd/pkg/logutil"
+
 	"github.com/coreos/pkg/capnslog"
 )
 
@@ -39,37 +38,18 @@ var (
 	mlog = logutil.NewMergeLogger(plog)
 )
 
-// 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)
-		}
-	case auth.Error:
+	if e, ok := err.(auth.Error); ok {
 		herr := httptypes.NewHTTPError(e.HTTPStatus(), e.Error())
 		if et := herr.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)
-		}
+		return
 	}
+	etcdhttp.WriteError(w, r, err)
 }
 
 // allowMethod verifies that the given method is one of the allowed methods,

+ 2 - 1
integration/cluster.go

@@ -38,6 +38,7 @@ import (
 	"github.com/coreos/etcd/clientv3"
 	"github.com/coreos/etcd/embed"
 	"github.com/coreos/etcd/etcdserver"
+	"github.com/coreos/etcd/etcdserver/api/etcdhttp"
 	"github.com/coreos/etcd/etcdserver/api/v2http"
 	"github.com/coreos/etcd/etcdserver/api/v3client"
 	"github.com/coreos/etcd/etcdserver/api/v3election"
@@ -648,7 +649,7 @@ func (m *member) Launch() error {
 	m.s.SyncTicker = time.NewTicker(500 * time.Millisecond)
 	m.s.Start()
 
-	m.raftHandler = &testutil.PauseableHandler{Next: v2http.NewPeerHandler(m.s)}
+	m.raftHandler = &testutil.PauseableHandler{Next: etcdhttp.NewPeerHandler(m.s)}
 
 	for _, ln := range m.PeerListeners {
 		hs := &httptest.Server{