client.go 23 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832
  1. // Copyright 2015 The etcd Authors
  2. //
  3. // Licensed under the Apache License, Version 2.0 (the "License");
  4. // you may not use this file except in compliance with the License.
  5. // You may obtain a copy of the License at
  6. //
  7. // http://www.apache.org/licenses/LICENSE-2.0
  8. //
  9. // Unless required by applicable law or agreed to in writing, software
  10. // distributed under the License is distributed on an "AS IS" BASIS,
  11. // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  12. // See the License for the specific language governing permissions and
  13. // limitations under the License.
  14. package v2http
  15. import (
  16. "encoding/json"
  17. "errors"
  18. "expvar"
  19. "fmt"
  20. "io/ioutil"
  21. "net/http"
  22. "net/http/pprof"
  23. "net/url"
  24. "path"
  25. "strconv"
  26. "strings"
  27. "time"
  28. etcdErr "github.com/coreos/etcd/error"
  29. "github.com/coreos/etcd/etcdserver"
  30. "github.com/coreos/etcd/etcdserver/api"
  31. "github.com/coreos/etcd/etcdserver/api/v2http/httptypes"
  32. "github.com/coreos/etcd/etcdserver/auth"
  33. "github.com/coreos/etcd/etcdserver/etcdserverpb"
  34. "github.com/coreos/etcd/etcdserver/membership"
  35. "github.com/coreos/etcd/etcdserver/stats"
  36. "github.com/coreos/etcd/pkg/types"
  37. "github.com/coreos/etcd/raft"
  38. "github.com/coreos/etcd/store"
  39. "github.com/coreos/etcd/version"
  40. "github.com/coreos/pkg/capnslog"
  41. "github.com/jonboulle/clockwork"
  42. "github.com/prometheus/client_golang/prometheus"
  43. "golang.org/x/net/context"
  44. )
  45. const (
  46. authPrefix = "/v2/auth"
  47. keysPrefix = "/v2/keys"
  48. deprecatedMachinesPrefix = "/v2/machines"
  49. membersPrefix = "/v2/members"
  50. statsPrefix = "/v2/stats"
  51. varsPath = "/debug/vars"
  52. metricsPath = "/metrics"
  53. healthPath = "/health"
  54. versionPath = "/version"
  55. configPath = "/config"
  56. pprofPrefix = "/debug/pprof"
  57. )
  58. // NewClientHandler generates a muxed http.Handler with the given parameters to serve etcd client requests.
  59. func NewClientHandler(server *etcdserver.EtcdServer, timeout time.Duration) http.Handler {
  60. sec := auth.NewStore(server, timeout)
  61. kh := &keysHandler{
  62. sec: sec,
  63. server: server,
  64. cluster: server.Cluster(),
  65. timer: server,
  66. timeout: timeout,
  67. clientCertAuthEnabled: server.Cfg.ClientCertAuthEnabled,
  68. }
  69. sh := &statsHandler{
  70. stats: server,
  71. }
  72. mh := &membersHandler{
  73. sec: sec,
  74. server: server,
  75. cluster: server.Cluster(),
  76. timeout: timeout,
  77. clock: clockwork.NewRealClock(),
  78. clientCertAuthEnabled: server.Cfg.ClientCertAuthEnabled,
  79. }
  80. dmh := &deprecatedMachinesHandler{
  81. cluster: server.Cluster(),
  82. }
  83. sech := &authHandler{
  84. sec: sec,
  85. cluster: server.Cluster(),
  86. clientCertAuthEnabled: server.Cfg.ClientCertAuthEnabled,
  87. }
  88. mux := http.NewServeMux()
  89. mux.HandleFunc("/", http.NotFound)
  90. mux.Handle(healthPath, healthHandler(server))
  91. mux.HandleFunc(versionPath, versionHandler(server.Cluster(), serveVersion))
  92. mux.Handle(keysPrefix, kh)
  93. mux.Handle(keysPrefix+"/", kh)
  94. mux.HandleFunc(statsPrefix+"/store", sh.serveStore)
  95. mux.HandleFunc(statsPrefix+"/self", sh.serveSelf)
  96. mux.HandleFunc(statsPrefix+"/leader", sh.serveLeader)
  97. mux.HandleFunc(varsPath, serveVars)
  98. mux.HandleFunc(configPath+"/local/log", logHandleFunc)
  99. mux.Handle(metricsPath, prometheus.Handler())
  100. mux.Handle(membersPrefix, mh)
  101. mux.Handle(membersPrefix+"/", mh)
  102. mux.Handle(deprecatedMachinesPrefix, dmh)
  103. handleAuth(mux, sech)
  104. if server.IsPprofEnabled() {
  105. plog.Infof("pprof is enabled under %s", pprofPrefix)
  106. mux.HandleFunc(pprofPrefix, pprof.Index)
  107. mux.HandleFunc(pprofPrefix+"/profile", pprof.Profile)
  108. mux.HandleFunc(pprofPrefix+"/symbol", pprof.Symbol)
  109. mux.HandleFunc(pprofPrefix+"/cmdline", pprof.Cmdline)
  110. // TODO: currently, we don't create an entry for pprof.Trace,
  111. // because go 1.4 doesn't provide it. After support of go 1.4 is dropped,
  112. // we should add the entry.
  113. mux.Handle(pprofPrefix+"/heap", pprof.Handler("heap"))
  114. mux.Handle(pprofPrefix+"/goroutine", pprof.Handler("goroutine"))
  115. mux.Handle(pprofPrefix+"/threadcreate", pprof.Handler("threadcreate"))
  116. mux.Handle(pprofPrefix+"/block", pprof.Handler("block"))
  117. }
  118. return requestLogger(mux)
  119. }
  120. type keysHandler struct {
  121. sec auth.Store
  122. server etcdserver.Server
  123. cluster api.Cluster
  124. timer etcdserver.RaftTimer
  125. timeout time.Duration
  126. clientCertAuthEnabled bool
  127. }
  128. func (h *keysHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
  129. if !allowMethod(w, r.Method, "HEAD", "GET", "PUT", "POST", "DELETE") {
  130. return
  131. }
  132. w.Header().Set("X-Etcd-Cluster-ID", h.cluster.ID().String())
  133. ctx, cancel := context.WithTimeout(context.Background(), h.timeout)
  134. defer cancel()
  135. clock := clockwork.NewRealClock()
  136. startTime := clock.Now()
  137. rr, err := parseKeyRequest(r, clock)
  138. if err != nil {
  139. writeKeyError(w, err)
  140. return
  141. }
  142. // The path must be valid at this point (we've parsed the request successfully).
  143. if !hasKeyPrefixAccess(h.sec, r, r.URL.Path[len(keysPrefix):], rr.Recursive, h.clientCertAuthEnabled) {
  144. writeKeyNoAuth(w)
  145. return
  146. }
  147. if !rr.Wait {
  148. reportRequestReceived(rr)
  149. }
  150. resp, err := h.server.Do(ctx, rr)
  151. if err != nil {
  152. err = trimErrorPrefix(err, etcdserver.StoreKeysPrefix)
  153. writeKeyError(w, err)
  154. reportRequestFailed(rr, err)
  155. return
  156. }
  157. switch {
  158. case resp.Event != nil:
  159. if err := writeKeyEvent(w, resp.Event, h.timer); err != nil {
  160. // Should never be reached
  161. plog.Errorf("error writing event (%v)", err)
  162. }
  163. reportRequestCompleted(rr, resp, startTime)
  164. case resp.Watcher != nil:
  165. ctx, cancel := context.WithTimeout(context.Background(), defaultWatchTimeout)
  166. defer cancel()
  167. handleKeyWatch(ctx, w, resp.Watcher, rr.Stream, h.timer)
  168. default:
  169. writeKeyError(w, errors.New("received response with no Event/Watcher!"))
  170. }
  171. }
  172. type deprecatedMachinesHandler struct {
  173. cluster api.Cluster
  174. }
  175. func (h *deprecatedMachinesHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
  176. if !allowMethod(w, r.Method, "GET", "HEAD") {
  177. return
  178. }
  179. endpoints := h.cluster.ClientURLs()
  180. w.Write([]byte(strings.Join(endpoints, ", ")))
  181. }
  182. type membersHandler struct {
  183. sec auth.Store
  184. server etcdserver.Server
  185. cluster api.Cluster
  186. timeout time.Duration
  187. clock clockwork.Clock
  188. clientCertAuthEnabled bool
  189. }
  190. func (h *membersHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
  191. if !allowMethod(w, r.Method, "GET", "POST", "DELETE", "PUT") {
  192. return
  193. }
  194. if !hasWriteRootAccess(h.sec, r, h.clientCertAuthEnabled) {
  195. writeNoAuth(w, r)
  196. return
  197. }
  198. w.Header().Set("X-Etcd-Cluster-ID", h.cluster.ID().String())
  199. ctx, cancel := context.WithTimeout(context.Background(), h.timeout)
  200. defer cancel()
  201. switch r.Method {
  202. case "GET":
  203. switch trimPrefix(r.URL.Path, membersPrefix) {
  204. case "":
  205. mc := newMemberCollection(h.cluster.Members())
  206. w.Header().Set("Content-Type", "application/json")
  207. if err := json.NewEncoder(w).Encode(mc); err != nil {
  208. plog.Warningf("failed to encode members response (%v)", err)
  209. }
  210. case "leader":
  211. id := h.server.Leader()
  212. if id == 0 {
  213. writeError(w, r, httptypes.NewHTTPError(http.StatusServiceUnavailable, "During election"))
  214. return
  215. }
  216. m := newMember(h.cluster.Member(id))
  217. w.Header().Set("Content-Type", "application/json")
  218. if err := json.NewEncoder(w).Encode(m); err != nil {
  219. plog.Warningf("failed to encode members response (%v)", err)
  220. }
  221. default:
  222. writeError(w, r, httptypes.NewHTTPError(http.StatusNotFound, "Not found"))
  223. }
  224. case "POST":
  225. req := httptypes.MemberCreateRequest{}
  226. if ok := unmarshalRequest(r, &req, w); !ok {
  227. return
  228. }
  229. now := h.clock.Now()
  230. m := membership.NewMember("", req.PeerURLs, "", &now)
  231. err := h.server.AddMember(ctx, *m)
  232. switch {
  233. case err == membership.ErrIDExists || err == membership.ErrPeerURLexists:
  234. writeError(w, r, httptypes.NewHTTPError(http.StatusConflict, err.Error()))
  235. return
  236. case err != nil:
  237. plog.Errorf("error adding member %s (%v)", m.ID, err)
  238. writeError(w, r, err)
  239. return
  240. }
  241. res := newMember(m)
  242. w.Header().Set("Content-Type", "application/json")
  243. w.WriteHeader(http.StatusCreated)
  244. if err := json.NewEncoder(w).Encode(res); err != nil {
  245. plog.Warningf("failed to encode members response (%v)", err)
  246. }
  247. case "DELETE":
  248. id, ok := getID(r.URL.Path, w)
  249. if !ok {
  250. return
  251. }
  252. err := h.server.RemoveMember(ctx, uint64(id))
  253. switch {
  254. case err == membership.ErrIDRemoved:
  255. writeError(w, r, httptypes.NewHTTPError(http.StatusGone, fmt.Sprintf("Member permanently removed: %s", id)))
  256. case err == membership.ErrIDNotFound:
  257. writeError(w, r, httptypes.NewHTTPError(http.StatusNotFound, fmt.Sprintf("No such member: %s", id)))
  258. case err != nil:
  259. plog.Errorf("error removing member %s (%v)", id, err)
  260. writeError(w, r, err)
  261. default:
  262. w.WriteHeader(http.StatusNoContent)
  263. }
  264. case "PUT":
  265. id, ok := getID(r.URL.Path, w)
  266. if !ok {
  267. return
  268. }
  269. req := httptypes.MemberUpdateRequest{}
  270. if ok := unmarshalRequest(r, &req, w); !ok {
  271. return
  272. }
  273. m := membership.Member{
  274. ID: id,
  275. RaftAttributes: membership.RaftAttributes{PeerURLs: req.PeerURLs.StringSlice()},
  276. }
  277. err := h.server.UpdateMember(ctx, m)
  278. switch {
  279. case err == membership.ErrPeerURLexists:
  280. writeError(w, r, httptypes.NewHTTPError(http.StatusConflict, err.Error()))
  281. case err == membership.ErrIDNotFound:
  282. writeError(w, r, httptypes.NewHTTPError(http.StatusNotFound, fmt.Sprintf("No such member: %s", id)))
  283. case err != nil:
  284. plog.Errorf("error updating member %s (%v)", m.ID, err)
  285. writeError(w, r, err)
  286. default:
  287. w.WriteHeader(http.StatusNoContent)
  288. }
  289. }
  290. }
  291. type statsHandler struct {
  292. stats stats.Stats
  293. }
  294. func (h *statsHandler) serveStore(w http.ResponseWriter, r *http.Request) {
  295. if !allowMethod(w, r.Method, "GET") {
  296. return
  297. }
  298. w.Header().Set("Content-Type", "application/json")
  299. w.Write(h.stats.StoreStats())
  300. }
  301. func (h *statsHandler) serveSelf(w http.ResponseWriter, r *http.Request) {
  302. if !allowMethod(w, r.Method, "GET") {
  303. return
  304. }
  305. w.Header().Set("Content-Type", "application/json")
  306. w.Write(h.stats.SelfStats())
  307. }
  308. func (h *statsHandler) serveLeader(w http.ResponseWriter, r *http.Request) {
  309. if !allowMethod(w, r.Method, "GET") {
  310. return
  311. }
  312. stats := h.stats.LeaderStats()
  313. if stats == nil {
  314. writeError(w, r, httptypes.NewHTTPError(http.StatusForbidden, "not current leader"))
  315. return
  316. }
  317. w.Header().Set("Content-Type", "application/json")
  318. w.Write(stats)
  319. }
  320. func serveVars(w http.ResponseWriter, r *http.Request) {
  321. if !allowMethod(w, r.Method, "GET") {
  322. return
  323. }
  324. w.Header().Set("Content-Type", "application/json; charset=utf-8")
  325. fmt.Fprintf(w, "{\n")
  326. first := true
  327. expvar.Do(func(kv expvar.KeyValue) {
  328. if !first {
  329. fmt.Fprintf(w, ",\n")
  330. }
  331. first = false
  332. fmt.Fprintf(w, "%q: %s", kv.Key, kv.Value)
  333. })
  334. fmt.Fprintf(w, "\n}\n")
  335. }
  336. // TODO: change etcdserver to raft interface when we have it.
  337. // add test for healthHandler when we have the interface ready.
  338. func healthHandler(server *etcdserver.EtcdServer) http.HandlerFunc {
  339. return func(w http.ResponseWriter, r *http.Request) {
  340. if !allowMethod(w, r.Method, "GET") {
  341. return
  342. }
  343. if uint64(server.Leader()) == raft.None {
  344. http.Error(w, `{"health": "false"}`, http.StatusServiceUnavailable)
  345. return
  346. }
  347. // wait for raft's progress
  348. index := server.Index()
  349. for i := 0; i < 3; i++ {
  350. time.Sleep(250 * time.Millisecond)
  351. if server.Index() > index {
  352. w.WriteHeader(http.StatusOK)
  353. w.Write([]byte(`{"health": "true"}`))
  354. return
  355. }
  356. }
  357. http.Error(w, `{"health": "false"}`, http.StatusServiceUnavailable)
  358. return
  359. }
  360. }
  361. func versionHandler(c api.Cluster, fn func(http.ResponseWriter, *http.Request, string)) http.HandlerFunc {
  362. return func(w http.ResponseWriter, r *http.Request) {
  363. v := c.Version()
  364. if v != nil {
  365. fn(w, r, v.String())
  366. } else {
  367. fn(w, r, "not_decided")
  368. }
  369. }
  370. }
  371. func serveVersion(w http.ResponseWriter, r *http.Request, clusterV string) {
  372. if !allowMethod(w, r.Method, "GET") {
  373. return
  374. }
  375. vs := version.Versions{
  376. Server: version.Version,
  377. Cluster: clusterV,
  378. }
  379. w.Header().Set("Content-Type", "application/json")
  380. b, err := json.Marshal(&vs)
  381. if err != nil {
  382. plog.Panicf("cannot marshal versions to json (%v)", err)
  383. }
  384. w.Write(b)
  385. }
  386. func logHandleFunc(w http.ResponseWriter, r *http.Request) {
  387. if !allowMethod(w, r.Method, "PUT") {
  388. return
  389. }
  390. in := struct{ Level string }{}
  391. d := json.NewDecoder(r.Body)
  392. if err := d.Decode(&in); err != nil {
  393. writeError(w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Invalid json body"))
  394. return
  395. }
  396. logl, err := capnslog.ParseLevel(strings.ToUpper(in.Level))
  397. if err != nil {
  398. writeError(w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Invalid log level "+in.Level))
  399. return
  400. }
  401. plog.Noticef("globalLogLevel set to %q", logl.String())
  402. capnslog.SetGlobalLogLevel(logl)
  403. w.WriteHeader(http.StatusNoContent)
  404. }
  405. // parseKeyRequest converts a received http.Request on keysPrefix to
  406. // a server Request, performing validation of supplied fields as appropriate.
  407. // If any validation fails, an empty Request and non-nil error is returned.
  408. func parseKeyRequest(r *http.Request, clock clockwork.Clock) (etcdserverpb.Request, error) {
  409. emptyReq := etcdserverpb.Request{}
  410. err := r.ParseForm()
  411. if err != nil {
  412. return emptyReq, etcdErr.NewRequestError(
  413. etcdErr.EcodeInvalidForm,
  414. err.Error(),
  415. )
  416. }
  417. if !strings.HasPrefix(r.URL.Path, keysPrefix) {
  418. return emptyReq, etcdErr.NewRequestError(
  419. etcdErr.EcodeInvalidForm,
  420. "incorrect key prefix",
  421. )
  422. }
  423. p := path.Join(etcdserver.StoreKeysPrefix, r.URL.Path[len(keysPrefix):])
  424. var pIdx, wIdx uint64
  425. if pIdx, err = getUint64(r.Form, "prevIndex"); err != nil {
  426. return emptyReq, etcdErr.NewRequestError(
  427. etcdErr.EcodeIndexNaN,
  428. `invalid value for "prevIndex"`,
  429. )
  430. }
  431. if wIdx, err = getUint64(r.Form, "waitIndex"); err != nil {
  432. return emptyReq, etcdErr.NewRequestError(
  433. etcdErr.EcodeIndexNaN,
  434. `invalid value for "waitIndex"`,
  435. )
  436. }
  437. var rec, sort, wait, dir, quorum, stream bool
  438. if rec, err = getBool(r.Form, "recursive"); err != nil {
  439. return emptyReq, etcdErr.NewRequestError(
  440. etcdErr.EcodeInvalidField,
  441. `invalid value for "recursive"`,
  442. )
  443. }
  444. if sort, err = getBool(r.Form, "sorted"); err != nil {
  445. return emptyReq, etcdErr.NewRequestError(
  446. etcdErr.EcodeInvalidField,
  447. `invalid value for "sorted"`,
  448. )
  449. }
  450. if wait, err = getBool(r.Form, "wait"); err != nil {
  451. return emptyReq, etcdErr.NewRequestError(
  452. etcdErr.EcodeInvalidField,
  453. `invalid value for "wait"`,
  454. )
  455. }
  456. // TODO(jonboulle): define what parameters dir is/isn't compatible with?
  457. if dir, err = getBool(r.Form, "dir"); err != nil {
  458. return emptyReq, etcdErr.NewRequestError(
  459. etcdErr.EcodeInvalidField,
  460. `invalid value for "dir"`,
  461. )
  462. }
  463. if quorum, err = getBool(r.Form, "quorum"); err != nil {
  464. return emptyReq, etcdErr.NewRequestError(
  465. etcdErr.EcodeInvalidField,
  466. `invalid value for "quorum"`,
  467. )
  468. }
  469. if stream, err = getBool(r.Form, "stream"); err != nil {
  470. return emptyReq, etcdErr.NewRequestError(
  471. etcdErr.EcodeInvalidField,
  472. `invalid value for "stream"`,
  473. )
  474. }
  475. if wait && r.Method != "GET" {
  476. return emptyReq, etcdErr.NewRequestError(
  477. etcdErr.EcodeInvalidField,
  478. `"wait" can only be used with GET requests`,
  479. )
  480. }
  481. pV := r.FormValue("prevValue")
  482. if _, ok := r.Form["prevValue"]; ok && pV == "" {
  483. return emptyReq, etcdErr.NewRequestError(
  484. etcdErr.EcodePrevValueRequired,
  485. `"prevValue" cannot be empty`,
  486. )
  487. }
  488. // TTL is nullable, so leave it null if not specified
  489. // or an empty string
  490. var ttl *uint64
  491. if len(r.FormValue("ttl")) > 0 {
  492. i, err := getUint64(r.Form, "ttl")
  493. if err != nil {
  494. return emptyReq, etcdErr.NewRequestError(
  495. etcdErr.EcodeTTLNaN,
  496. `invalid value for "ttl"`,
  497. )
  498. }
  499. ttl = &i
  500. }
  501. // prevExist is nullable, so leave it null if not specified
  502. var pe *bool
  503. if _, ok := r.Form["prevExist"]; ok {
  504. bv, err := getBool(r.Form, "prevExist")
  505. if err != nil {
  506. return emptyReq, etcdErr.NewRequestError(
  507. etcdErr.EcodeInvalidField,
  508. "invalid value for prevExist",
  509. )
  510. }
  511. pe = &bv
  512. }
  513. // refresh is nullable, so leave it null if not specified
  514. var refresh *bool
  515. if _, ok := r.Form["refresh"]; ok {
  516. bv, err := getBool(r.Form, "refresh")
  517. if err != nil {
  518. return emptyReq, etcdErr.NewRequestError(
  519. etcdErr.EcodeInvalidField,
  520. "invalid value for refresh",
  521. )
  522. }
  523. refresh = &bv
  524. if refresh != nil && *refresh {
  525. val := r.FormValue("value")
  526. if _, ok := r.Form["value"]; ok && val != "" {
  527. return emptyReq, etcdErr.NewRequestError(
  528. etcdErr.EcodeRefreshValue,
  529. `A value was provided on a refresh`,
  530. )
  531. }
  532. if ttl == nil {
  533. return emptyReq, etcdErr.NewRequestError(
  534. etcdErr.EcodeRefreshTTLRequired,
  535. `No TTL value set`,
  536. )
  537. }
  538. }
  539. }
  540. rr := etcdserverpb.Request{
  541. Method: r.Method,
  542. Path: p,
  543. Val: r.FormValue("value"),
  544. Dir: dir,
  545. PrevValue: pV,
  546. PrevIndex: pIdx,
  547. PrevExist: pe,
  548. Wait: wait,
  549. Since: wIdx,
  550. Recursive: rec,
  551. Sorted: sort,
  552. Quorum: quorum,
  553. Stream: stream,
  554. }
  555. if pe != nil {
  556. rr.PrevExist = pe
  557. }
  558. if refresh != nil {
  559. rr.Refresh = refresh
  560. }
  561. // Null TTL is equivalent to unset Expiration
  562. if ttl != nil {
  563. expr := time.Duration(*ttl) * time.Second
  564. rr.Expiration = clock.Now().Add(expr).UnixNano()
  565. }
  566. return rr, nil
  567. }
  568. // writeKeyEvent trims the prefix of key path in a single Event under
  569. // StoreKeysPrefix, serializes it and writes the resulting JSON to the given
  570. // ResponseWriter, along with the appropriate headers.
  571. func writeKeyEvent(w http.ResponseWriter, ev *store.Event, rt etcdserver.RaftTimer) error {
  572. if ev == nil {
  573. return errors.New("cannot write empty Event!")
  574. }
  575. w.Header().Set("Content-Type", "application/json")
  576. w.Header().Set("X-Etcd-Index", fmt.Sprint(ev.EtcdIndex))
  577. w.Header().Set("X-Raft-Index", fmt.Sprint(rt.Index()))
  578. w.Header().Set("X-Raft-Term", fmt.Sprint(rt.Term()))
  579. if ev.IsCreated() {
  580. w.WriteHeader(http.StatusCreated)
  581. }
  582. ev = trimEventPrefix(ev, etcdserver.StoreKeysPrefix)
  583. return json.NewEncoder(w).Encode(ev)
  584. }
  585. func writeKeyNoAuth(w http.ResponseWriter) {
  586. e := etcdErr.NewError(etcdErr.EcodeUnauthorized, "Insufficient credentials", 0)
  587. e.WriteTo(w)
  588. }
  589. // writeKeyError logs and writes the given Error to the ResponseWriter.
  590. // If Error is not an etcdErr, the error will be converted to an etcd error.
  591. func writeKeyError(w http.ResponseWriter, err error) {
  592. if err == nil {
  593. return
  594. }
  595. switch e := err.(type) {
  596. case *etcdErr.Error:
  597. e.WriteTo(w)
  598. default:
  599. switch err {
  600. case etcdserver.ErrTimeoutDueToLeaderFail, etcdserver.ErrTimeoutDueToConnectionLost:
  601. mlog.MergeError(err)
  602. default:
  603. mlog.MergeErrorf("got unexpected response error (%v)", err)
  604. }
  605. ee := etcdErr.NewError(etcdErr.EcodeRaftInternal, err.Error(), 0)
  606. ee.WriteTo(w)
  607. }
  608. }
  609. func handleKeyWatch(ctx context.Context, w http.ResponseWriter, wa store.Watcher, stream bool, rt etcdserver.RaftTimer) {
  610. defer wa.Remove()
  611. ech := wa.EventChan()
  612. var nch <-chan bool
  613. if x, ok := w.(http.CloseNotifier); ok {
  614. nch = x.CloseNotify()
  615. }
  616. w.Header().Set("Content-Type", "application/json")
  617. w.Header().Set("X-Etcd-Index", fmt.Sprint(wa.StartIndex()))
  618. w.Header().Set("X-Raft-Index", fmt.Sprint(rt.Index()))
  619. w.Header().Set("X-Raft-Term", fmt.Sprint(rt.Term()))
  620. w.WriteHeader(http.StatusOK)
  621. // Ensure headers are flushed early, in case of long polling
  622. w.(http.Flusher).Flush()
  623. for {
  624. select {
  625. case <-nch:
  626. // Client closed connection. Nothing to do.
  627. return
  628. case <-ctx.Done():
  629. // Timed out. net/http will close the connection for us, so nothing to do.
  630. return
  631. case ev, ok := <-ech:
  632. if !ok {
  633. // If the channel is closed this may be an indication of
  634. // that notifications are much more than we are able to
  635. // send to the client in time. Then we simply end streaming.
  636. return
  637. }
  638. ev = trimEventPrefix(ev, etcdserver.StoreKeysPrefix)
  639. if err := json.NewEncoder(w).Encode(ev); err != nil {
  640. // Should never be reached
  641. plog.Warningf("error writing event (%v)", err)
  642. return
  643. }
  644. if !stream {
  645. return
  646. }
  647. w.(http.Flusher).Flush()
  648. }
  649. }
  650. }
  651. func trimEventPrefix(ev *store.Event, prefix string) *store.Event {
  652. if ev == nil {
  653. return nil
  654. }
  655. // Since the *Event may reference one in the store history
  656. // history, we must copy it before modifying
  657. e := ev.Clone()
  658. trimNodeExternPrefix(e.Node, prefix)
  659. trimNodeExternPrefix(e.PrevNode, prefix)
  660. return e
  661. }
  662. func trimNodeExternPrefix(n *store.NodeExtern, prefix string) {
  663. if n == nil {
  664. return
  665. }
  666. n.Key = strings.TrimPrefix(n.Key, prefix)
  667. for _, nn := range n.Nodes {
  668. trimNodeExternPrefix(nn, prefix)
  669. }
  670. }
  671. func trimErrorPrefix(err error, prefix string) error {
  672. if e, ok := err.(*etcdErr.Error); ok {
  673. e.Cause = strings.TrimPrefix(e.Cause, prefix)
  674. }
  675. return err
  676. }
  677. func unmarshalRequest(r *http.Request, req json.Unmarshaler, w http.ResponseWriter) bool {
  678. ctype := r.Header.Get("Content-Type")
  679. if ctype != "application/json" {
  680. writeError(w, r, httptypes.NewHTTPError(http.StatusUnsupportedMediaType, fmt.Sprintf("Bad Content-Type %s, accept application/json", ctype)))
  681. return false
  682. }
  683. b, err := ioutil.ReadAll(r.Body)
  684. if err != nil {
  685. writeError(w, r, httptypes.NewHTTPError(http.StatusBadRequest, err.Error()))
  686. return false
  687. }
  688. if err := req.UnmarshalJSON(b); err != nil {
  689. writeError(w, r, httptypes.NewHTTPError(http.StatusBadRequest, err.Error()))
  690. return false
  691. }
  692. return true
  693. }
  694. func getID(p string, w http.ResponseWriter) (types.ID, bool) {
  695. idStr := trimPrefix(p, membersPrefix)
  696. if idStr == "" {
  697. http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
  698. return 0, false
  699. }
  700. id, err := types.IDFromString(idStr)
  701. if err != nil {
  702. writeError(w, nil, httptypes.NewHTTPError(http.StatusNotFound, fmt.Sprintf("No such member: %s", idStr)))
  703. return 0, false
  704. }
  705. return id, true
  706. }
  707. // getUint64 extracts a uint64 by the given key from a Form. If the key does
  708. // not exist in the form, 0 is returned. If the key exists but the value is
  709. // badly formed, an error is returned. If multiple values are present only the
  710. // first is considered.
  711. func getUint64(form url.Values, key string) (i uint64, err error) {
  712. if vals, ok := form[key]; ok {
  713. i, err = strconv.ParseUint(vals[0], 10, 64)
  714. }
  715. return
  716. }
  717. // getBool extracts a bool by the given key from a Form. If the key does not
  718. // exist in the form, false is returned. If the key exists but the value is
  719. // badly formed, an error is returned. If multiple values are present only the
  720. // first is considered.
  721. func getBool(form url.Values, key string) (b bool, err error) {
  722. if vals, ok := form[key]; ok {
  723. b, err = strconv.ParseBool(vals[0])
  724. }
  725. return
  726. }
  727. // trimPrefix removes a given prefix and any slash following the prefix
  728. // e.g.: trimPrefix("foo", "foo") == trimPrefix("foo/", "foo") == ""
  729. func trimPrefix(p, prefix string) (s string) {
  730. s = strings.TrimPrefix(p, prefix)
  731. s = strings.TrimPrefix(s, "/")
  732. return
  733. }
  734. func newMemberCollection(ms []*membership.Member) *httptypes.MemberCollection {
  735. c := httptypes.MemberCollection(make([]httptypes.Member, len(ms)))
  736. for i, m := range ms {
  737. c[i] = newMember(m)
  738. }
  739. return &c
  740. }
  741. func newMember(m *membership.Member) httptypes.Member {
  742. tm := httptypes.Member{
  743. ID: m.ID.String(),
  744. Name: m.Name,
  745. PeerURLs: make([]string, len(m.PeerURLs)),
  746. ClientURLs: make([]string, len(m.ClientURLs)),
  747. }
  748. copy(tm.PeerURLs, m.PeerURLs)
  749. copy(tm.ClientURLs, m.ClientURLs)
  750. return tm
  751. }