123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832 |
- // 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 v2http
- import (
- "encoding/json"
- "errors"
- "expvar"
- "fmt"
- "io/ioutil"
- "net/http"
- "net/http/pprof"
- "net/url"
- "path"
- "strconv"
- "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/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"
- )
- const (
- authPrefix = "/v2/auth"
- keysPrefix = "/v2/keys"
- deprecatedMachinesPrefix = "/v2/machines"
- membersPrefix = "/v2/members"
- statsPrefix = "/v2/stats"
- varsPath = "/debug/vars"
- metricsPath = "/metrics"
- healthPath = "/health"
- versionPath = "/version"
- configPath = "/config"
- pprofPrefix = "/debug/pprof"
- )
- // 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)
- kh := &keysHandler{
- sec: sec,
- server: server,
- cluster: server.Cluster(),
- timer: server,
- timeout: timeout,
- clientCertAuthEnabled: server.Cfg.ClientCertAuthEnabled,
- }
- sh := &statsHandler{
- stats: server,
- }
- mh := &membersHandler{
- sec: sec,
- server: server,
- cluster: server.Cluster(),
- timeout: timeout,
- clock: clockwork.NewRealClock(),
- clientCertAuthEnabled: server.Cfg.ClientCertAuthEnabled,
- }
- dmh := &deprecatedMachinesHandler{
- cluster: server.Cluster(),
- }
- sech := &authHandler{
- sec: sec,
- 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(deprecatedMachinesPrefix, dmh)
- handleAuth(mux, sech)
- if server.IsPprofEnabled() {
- plog.Infof("pprof is enabled under %s", pprofPrefix)
- mux.HandleFunc(pprofPrefix, pprof.Index)
- mux.HandleFunc(pprofPrefix+"/profile", pprof.Profile)
- mux.HandleFunc(pprofPrefix+"/symbol", pprof.Symbol)
- mux.HandleFunc(pprofPrefix+"/cmdline", pprof.Cmdline)
- // TODO: currently, we don't create an entry for pprof.Trace,
- // because go 1.4 doesn't provide it. After support of go 1.4 is dropped,
- // we should add the entry.
- mux.Handle(pprofPrefix+"/heap", pprof.Handler("heap"))
- mux.Handle(pprofPrefix+"/goroutine", pprof.Handler("goroutine"))
- mux.Handle(pprofPrefix+"/threadcreate", pprof.Handler("threadcreate"))
- mux.Handle(pprofPrefix+"/block", pprof.Handler("block"))
- }
- return requestLogger(mux)
- }
- type keysHandler struct {
- sec auth.Store
- server etcdserver.Server
- cluster api.Cluster
- timer etcdserver.RaftTimer
- timeout time.Duration
- clientCertAuthEnabled bool
- }
- func (h *keysHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
- if !allowMethod(w, r.Method, "HEAD", "GET", "PUT", "POST", "DELETE") {
- return
- }
- w.Header().Set("X-Etcd-Cluster-ID", h.cluster.ID().String())
- ctx, cancel := context.WithTimeout(context.Background(), h.timeout)
- defer cancel()
- clock := clockwork.NewRealClock()
- startTime := clock.Now()
- rr, err := parseKeyRequest(r, clock)
- if err != nil {
- writeKeyError(w, err)
- return
- }
- // The path must be valid at this point (we've parsed the request successfully).
- if !hasKeyPrefixAccess(h.sec, r, r.URL.Path[len(keysPrefix):], rr.Recursive, h.clientCertAuthEnabled) {
- 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 {
- case resp.Event != nil:
- if err := writeKeyEvent(w, resp.Event, h.timer); err != nil {
- // 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()
- handleKeyWatch(ctx, w, resp.Watcher, rr.Stream, h.timer)
- default:
- writeKeyError(w, errors.New("received response with no Event/Watcher!"))
- }
- }
- type deprecatedMachinesHandler struct {
- cluster api.Cluster
- }
- func (h *deprecatedMachinesHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
- if !allowMethod(w, r.Method, "GET", "HEAD") {
- return
- }
- endpoints := h.cluster.ClientURLs()
- w.Write([]byte(strings.Join(endpoints, ", ")))
- }
- type membersHandler struct {
- sec auth.Store
- server etcdserver.Server
- cluster api.Cluster
- timeout time.Duration
- clock clockwork.Clock
- clientCertAuthEnabled bool
- }
- func (h *membersHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
- if !allowMethod(w, r.Method, "GET", "POST", "DELETE", "PUT") {
- return
- }
- if !hasWriteRootAccess(h.sec, r, h.clientCertAuthEnabled) {
- writeNoAuth(w, r)
- return
- }
- w.Header().Set("X-Etcd-Cluster-ID", h.cluster.ID().String())
- ctx, cancel := context.WithTimeout(context.Background(), h.timeout)
- defer cancel()
- switch r.Method {
- case "GET":
- switch trimPrefix(r.URL.Path, membersPrefix) {
- case "":
- mc := newMemberCollection(h.cluster.Members())
- w.Header().Set("Content-Type", "application/json")
- if err := json.NewEncoder(w).Encode(mc); err != nil {
- plog.Warningf("failed to encode members response (%v)", err)
- }
- case "leader":
- id := h.server.Leader()
- if id == 0 {
- writeError(w, r, httptypes.NewHTTPError(http.StatusServiceUnavailable, "During election"))
- return
- }
- m := newMember(h.cluster.Member(id))
- w.Header().Set("Content-Type", "application/json")
- if err := json.NewEncoder(w).Encode(m); err != nil {
- plog.Warningf("failed to encode members response (%v)", err)
- }
- default:
- writeError(w, r, httptypes.NewHTTPError(http.StatusNotFound, "Not found"))
- }
- case "POST":
- req := httptypes.MemberCreateRequest{}
- if ok := unmarshalRequest(r, &req, w); !ok {
- return
- }
- now := h.clock.Now()
- m := membership.NewMember("", req.PeerURLs, "", &now)
- err := h.server.AddMember(ctx, *m)
- switch {
- case err == membership.ErrIDExists || err == membership.ErrPeerURLexists:
- writeError(w, r, httptypes.NewHTTPError(http.StatusConflict, err.Error()))
- return
- case err != nil:
- plog.Errorf("error adding member %s (%v)", m.ID, err)
- writeError(w, r, err)
- return
- }
- res := newMember(m)
- w.Header().Set("Content-Type", "application/json")
- w.WriteHeader(http.StatusCreated)
- if err := json.NewEncoder(w).Encode(res); err != nil {
- plog.Warningf("failed to encode members response (%v)", err)
- }
- case "DELETE":
- id, ok := getID(r.URL.Path, w)
- if !ok {
- return
- }
- err := h.server.RemoveMember(ctx, uint64(id))
- switch {
- case err == membership.ErrIDRemoved:
- writeError(w, r, httptypes.NewHTTPError(http.StatusGone, fmt.Sprintf("Member permanently removed: %s", id)))
- case err == membership.ErrIDNotFound:
- writeError(w, r, httptypes.NewHTTPError(http.StatusNotFound, fmt.Sprintf("No such member: %s", id)))
- case err != nil:
- plog.Errorf("error removing member %s (%v)", id, err)
- writeError(w, r, err)
- default:
- w.WriteHeader(http.StatusNoContent)
- }
- case "PUT":
- id, ok := getID(r.URL.Path, w)
- if !ok {
- return
- }
- req := httptypes.MemberUpdateRequest{}
- if ok := unmarshalRequest(r, &req, w); !ok {
- return
- }
- m := membership.Member{
- ID: id,
- RaftAttributes: membership.RaftAttributes{PeerURLs: req.PeerURLs.StringSlice()},
- }
- err := h.server.UpdateMember(ctx, m)
- switch {
- case err == membership.ErrPeerURLexists:
- writeError(w, r, httptypes.NewHTTPError(http.StatusConflict, err.Error()))
- case err == membership.ErrIDNotFound:
- writeError(w, r, httptypes.NewHTTPError(http.StatusNotFound, fmt.Sprintf("No such member: %s", id)))
- case err != nil:
- plog.Errorf("error updating member %s (%v)", m.ID, err)
- writeError(w, r, err)
- default:
- w.WriteHeader(http.StatusNoContent)
- }
- }
- }
- type statsHandler struct {
- stats stats.Stats
- }
- func (h *statsHandler) serveStore(w http.ResponseWriter, r *http.Request) {
- if !allowMethod(w, r.Method, "GET") {
- return
- }
- w.Header().Set("Content-Type", "application/json")
- w.Write(h.stats.StoreStats())
- }
- func (h *statsHandler) serveSelf(w http.ResponseWriter, r *http.Request) {
- if !allowMethod(w, r.Method, "GET") {
- return
- }
- w.Header().Set("Content-Type", "application/json")
- w.Write(h.stats.SelfStats())
- }
- func (h *statsHandler) serveLeader(w http.ResponseWriter, r *http.Request) {
- if !allowMethod(w, r.Method, "GET") {
- return
- }
- stats := h.stats.LeaderStats()
- if stats == nil {
- 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")
- }
- // TODO: change etcdserver to raft interface when we have it.
- // add test for healthHandler when we have the interface ready.
- 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
- }
- // wait for raft's progress
- index := server.Index()
- for i := 0; i < 3; i++ {
- time.Sleep(250 * time.Millisecond)
- if server.Index() > index {
- w.WriteHeader(http.StatusOK)
- w.Write([]byte(`{"health": "true"}`))
- return
- }
- }
- http.Error(w, `{"health": "false"}`, http.StatusServiceUnavailable)
- return
- }
- }
- 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.
- func parseKeyRequest(r *http.Request, clock clockwork.Clock) (etcdserverpb.Request, error) {
- emptyReq := etcdserverpb.Request{}
- err := r.ParseForm()
- if err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeInvalidForm,
- err.Error(),
- )
- }
- if !strings.HasPrefix(r.URL.Path, keysPrefix) {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeInvalidForm,
- "incorrect key prefix",
- )
- }
- p := path.Join(etcdserver.StoreKeysPrefix, r.URL.Path[len(keysPrefix):])
- var pIdx, wIdx uint64
- if pIdx, err = getUint64(r.Form, "prevIndex"); err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeIndexNaN,
- `invalid value for "prevIndex"`,
- )
- }
- if wIdx, err = getUint64(r.Form, "waitIndex"); err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeIndexNaN,
- `invalid value for "waitIndex"`,
- )
- }
- var rec, sort, wait, dir, quorum, stream bool
- if rec, err = getBool(r.Form, "recursive"); err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeInvalidField,
- `invalid value for "recursive"`,
- )
- }
- if sort, err = getBool(r.Form, "sorted"); err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeInvalidField,
- `invalid value for "sorted"`,
- )
- }
- if wait, err = getBool(r.Form, "wait"); err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeInvalidField,
- `invalid value for "wait"`,
- )
- }
- // TODO(jonboulle): define what parameters dir is/isn't compatible with?
- if dir, err = getBool(r.Form, "dir"); err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeInvalidField,
- `invalid value for "dir"`,
- )
- }
- if quorum, err = getBool(r.Form, "quorum"); err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeInvalidField,
- `invalid value for "quorum"`,
- )
- }
- if stream, err = getBool(r.Form, "stream"); err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeInvalidField,
- `invalid value for "stream"`,
- )
- }
- if wait && r.Method != "GET" {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeInvalidField,
- `"wait" can only be used with GET requests`,
- )
- }
- pV := r.FormValue("prevValue")
- if _, ok := r.Form["prevValue"]; ok && pV == "" {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodePrevValueRequired,
- `"prevValue" cannot be empty`,
- )
- }
- // TTL is nullable, so leave it null if not specified
- // or an empty string
- var ttl *uint64
- if len(r.FormValue("ttl")) > 0 {
- i, err := getUint64(r.Form, "ttl")
- if err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeTTLNaN,
- `invalid value for "ttl"`,
- )
- }
- ttl = &i
- }
- // prevExist is nullable, so leave it null if not specified
- var pe *bool
- if _, ok := r.Form["prevExist"]; ok {
- bv, err := getBool(r.Form, "prevExist")
- if err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeInvalidField,
- "invalid value for prevExist",
- )
- }
- pe = &bv
- }
- // refresh is nullable, so leave it null if not specified
- var refresh *bool
- if _, ok := r.Form["refresh"]; ok {
- bv, err := getBool(r.Form, "refresh")
- if err != nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeInvalidField,
- "invalid value for refresh",
- )
- }
- refresh = &bv
- if refresh != nil && *refresh {
- val := r.FormValue("value")
- if _, ok := r.Form["value"]; ok && val != "" {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeRefreshValue,
- `A value was provided on a refresh`,
- )
- }
- if ttl == nil {
- return emptyReq, etcdErr.NewRequestError(
- etcdErr.EcodeRefreshTTLRequired,
- `No TTL value set`,
- )
- }
- }
- }
- rr := etcdserverpb.Request{
- Method: r.Method,
- Path: p,
- Val: r.FormValue("value"),
- Dir: dir,
- PrevValue: pV,
- PrevIndex: pIdx,
- PrevExist: pe,
- Wait: wait,
- Since: wIdx,
- Recursive: rec,
- Sorted: sort,
- Quorum: quorum,
- Stream: stream,
- }
- if pe != nil {
- rr.PrevExist = pe
- }
- if refresh != nil {
- rr.Refresh = refresh
- }
- // Null TTL is equivalent to unset Expiration
- if ttl != nil {
- expr := time.Duration(*ttl) * time.Second
- rr.Expiration = clock.Now().Add(expr).UnixNano()
- }
- return rr, nil
- }
- // writeKeyEvent trims the prefix of key path in a single Event under
- // StoreKeysPrefix, serializes it and writes the resulting JSON to the given
- // ResponseWriter, along with the appropriate headers.
- func writeKeyEvent(w http.ResponseWriter, ev *store.Event, rt etcdserver.RaftTimer) error {
- if ev == nil {
- return errors.New("cannot write empty Event!")
- }
- w.Header().Set("Content-Type", "application/json")
- w.Header().Set("X-Etcd-Index", fmt.Sprint(ev.EtcdIndex))
- w.Header().Set("X-Raft-Index", fmt.Sprint(rt.Index()))
- w.Header().Set("X-Raft-Term", fmt.Sprint(rt.Term()))
- if ev.IsCreated() {
- w.WriteHeader(http.StatusCreated)
- }
- ev = trimEventPrefix(ev, etcdserver.StoreKeysPrefix)
- return json.NewEncoder(w).Encode(ev)
- }
- func writeKeyNoAuth(w http.ResponseWriter) {
- e := etcdErr.NewError(etcdErr.EcodeUnauthorized, "Insufficient credentials", 0)
- e.WriteTo(w)
- }
- // writeKeyError logs and writes the given Error to the ResponseWriter.
- // If Error is not an etcdErr, the error will be converted to an etcd error.
- func writeKeyError(w http.ResponseWriter, err error) {
- if err == nil {
- return
- }
- switch e := err.(type) {
- case *etcdErr.Error:
- e.WriteTo(w)
- default:
- switch err {
- case etcdserver.ErrTimeoutDueToLeaderFail, etcdserver.ErrTimeoutDueToConnectionLost:
- mlog.MergeError(err)
- default:
- mlog.MergeErrorf("got unexpected response error (%v)", err)
- }
- ee := etcdErr.NewError(etcdErr.EcodeRaftInternal, err.Error(), 0)
- ee.WriteTo(w)
- }
- }
- func handleKeyWatch(ctx context.Context, w http.ResponseWriter, wa store.Watcher, stream bool, rt etcdserver.RaftTimer) {
- defer wa.Remove()
- ech := wa.EventChan()
- var nch <-chan bool
- if x, ok := w.(http.CloseNotifier); ok {
- nch = x.CloseNotify()
- }
- w.Header().Set("Content-Type", "application/json")
- w.Header().Set("X-Etcd-Index", fmt.Sprint(wa.StartIndex()))
- w.Header().Set("X-Raft-Index", fmt.Sprint(rt.Index()))
- w.Header().Set("X-Raft-Term", fmt.Sprint(rt.Term()))
- w.WriteHeader(http.StatusOK)
- // Ensure headers are flushed early, in case of long polling
- w.(http.Flusher).Flush()
- for {
- select {
- case <-nch:
- // Client closed connection. Nothing to do.
- return
- case <-ctx.Done():
- // Timed out. net/http will close the connection for us, so nothing to do.
- return
- case ev, ok := <-ech:
- if !ok {
- // If the channel is closed this may be an indication of
- // that notifications are much more than we are able to
- // send to the client in time. Then we simply end streaming.
- return
- }
- ev = trimEventPrefix(ev, etcdserver.StoreKeysPrefix)
- if err := json.NewEncoder(w).Encode(ev); err != nil {
- // Should never be reached
- plog.Warningf("error writing event (%v)", err)
- return
- }
- if !stream {
- return
- }
- w.(http.Flusher).Flush()
- }
- }
- }
- func trimEventPrefix(ev *store.Event, prefix string) *store.Event {
- if ev == nil {
- return nil
- }
- // Since the *Event may reference one in the store history
- // history, we must copy it before modifying
- e := ev.Clone()
- trimNodeExternPrefix(e.Node, prefix)
- trimNodeExternPrefix(e.PrevNode, prefix)
- return e
- }
- func trimNodeExternPrefix(n *store.NodeExtern, prefix string) {
- if n == nil {
- return
- }
- n.Key = strings.TrimPrefix(n.Key, prefix)
- for _, nn := range n.Nodes {
- trimNodeExternPrefix(nn, prefix)
- }
- }
- func trimErrorPrefix(err error, prefix string) error {
- if e, ok := err.(*etcdErr.Error); ok {
- e.Cause = strings.TrimPrefix(e.Cause, prefix)
- }
- return err
- }
- func unmarshalRequest(r *http.Request, req json.Unmarshaler, w http.ResponseWriter) bool {
- ctype := r.Header.Get("Content-Type")
- if ctype != "application/json" {
- writeError(w, r, httptypes.NewHTTPError(http.StatusUnsupportedMediaType, fmt.Sprintf("Bad Content-Type %s, accept application/json", ctype)))
- return false
- }
- b, err := ioutil.ReadAll(r.Body)
- if err != nil {
- writeError(w, r, httptypes.NewHTTPError(http.StatusBadRequest, err.Error()))
- return false
- }
- if err := req.UnmarshalJSON(b); err != nil {
- writeError(w, r, httptypes.NewHTTPError(http.StatusBadRequest, err.Error()))
- return false
- }
- return true
- }
- func getID(p string, w http.ResponseWriter) (types.ID, bool) {
- idStr := trimPrefix(p, membersPrefix)
- if idStr == "" {
- http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
- return 0, false
- }
- id, err := types.IDFromString(idStr)
- if err != nil {
- writeError(w, nil, httptypes.NewHTTPError(http.StatusNotFound, fmt.Sprintf("No such member: %s", idStr)))
- return 0, false
- }
- return id, true
- }
- // getUint64 extracts a uint64 by the given key from a Form. If the key does
- // not exist in the form, 0 is returned. If the key exists but the value is
- // badly formed, an error is returned. If multiple values are present only the
- // first is considered.
- func getUint64(form url.Values, key string) (i uint64, err error) {
- if vals, ok := form[key]; ok {
- i, err = strconv.ParseUint(vals[0], 10, 64)
- }
- return
- }
- // getBool extracts a bool by the given key from a Form. If the key does not
- // exist in the form, false is returned. If the key exists but the value is
- // badly formed, an error is returned. If multiple values are present only the
- // first is considered.
- func getBool(form url.Values, key string) (b bool, err error) {
- if vals, ok := form[key]; ok {
- b, err = strconv.ParseBool(vals[0])
- }
- return
- }
- // trimPrefix removes a given prefix and any slash following the prefix
- // e.g.: trimPrefix("foo", "foo") == trimPrefix("foo/", "foo") == ""
- func trimPrefix(p, prefix string) (s string) {
- s = strings.TrimPrefix(p, prefix)
- s = strings.TrimPrefix(s, "/")
- return
- }
- func newMemberCollection(ms []*membership.Member) *httptypes.MemberCollection {
- c := httptypes.MemberCollection(make([]httptypes.Member, len(ms)))
- for i, m := range ms {
- c[i] = newMember(m)
- }
- return &c
- }
- func newMember(m *membership.Member) httptypes.Member {
- tm := httptypes.Member{
- ID: m.ID.String(),
- Name: m.Name,
- PeerURLs: make([]string, len(m.PeerURLs)),
- ClientURLs: make([]string, len(m.ClientURLs)),
- }
- copy(tm.PeerURLs, m.PeerURLs)
- copy(tm.ClientURLs, m.ClientURLs)
- return tm
- }
|