client.go 23 KB

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