client.go 15 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557
  1. /*
  2. Copyright 2014 CoreOS, Inc.
  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. http://www.apache.org/licenses/LICENSE-2.0
  7. Unless required by applicable law or agreed to in writing, software
  8. distributed under the License is distributed on an "AS IS" BASIS,
  9. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  10. See the License for the specific language governing permissions and
  11. limitations under the License.
  12. */
  13. package etcdhttp
  14. import (
  15. "encoding/json"
  16. "errors"
  17. "fmt"
  18. "io/ioutil"
  19. "log"
  20. "net/http"
  21. "net/url"
  22. "path"
  23. "strconv"
  24. "strings"
  25. "time"
  26. "github.com/coreos/etcd/Godeps/_workspace/src/code.google.com/p/go.net/context"
  27. "github.com/coreos/etcd/Godeps/_workspace/src/github.com/jonboulle/clockwork"
  28. etcdErr "github.com/coreos/etcd/error"
  29. "github.com/coreos/etcd/etcdserver"
  30. "github.com/coreos/etcd/etcdserver/etcdhttp/httptypes"
  31. "github.com/coreos/etcd/etcdserver/etcdserverpb"
  32. "github.com/coreos/etcd/pkg/types"
  33. "github.com/coreos/etcd/store"
  34. "github.com/coreos/etcd/version"
  35. )
  36. const (
  37. keysPrefix = "/v2/keys"
  38. deprecatedMachinesPrefix = "/v2/machines"
  39. membersPrefix = "/v2/members"
  40. statsPrefix = "/v2/stats"
  41. versionPrefix = "/version"
  42. )
  43. // NewClientHandler generates a muxed http.Handler with the given parameters to serve etcd client requests.
  44. func NewClientHandler(server *etcdserver.EtcdServer) http.Handler {
  45. kh := &keysHandler{
  46. server: server,
  47. clusterInfo: server.Cluster,
  48. timer: server,
  49. timeout: defaultServerTimeout,
  50. }
  51. sh := &statsHandler{
  52. stats: server,
  53. }
  54. mh := &membersHandler{
  55. server: server,
  56. clusterInfo: server.Cluster,
  57. clock: clockwork.NewRealClock(),
  58. }
  59. dmh := &deprecatedMachinesHandler{
  60. clusterInfo: server.Cluster,
  61. }
  62. mux := http.NewServeMux()
  63. mux.HandleFunc("/", http.NotFound)
  64. mux.HandleFunc(versionPrefix, serveVersion)
  65. mux.Handle(keysPrefix, kh)
  66. mux.Handle(keysPrefix+"/", kh)
  67. mux.HandleFunc(statsPrefix+"/store", sh.serveStore)
  68. mux.HandleFunc(statsPrefix+"/self", sh.serveSelf)
  69. mux.HandleFunc(statsPrefix+"/leader", sh.serveLeader)
  70. mux.Handle(membersPrefix, mh)
  71. mux.Handle(membersPrefix+"/", mh)
  72. mux.Handle(deprecatedMachinesPrefix, dmh)
  73. return mux
  74. }
  75. type keysHandler struct {
  76. server etcdserver.Server
  77. clusterInfo etcdserver.ClusterInfo
  78. timer etcdserver.RaftTimer
  79. timeout time.Duration
  80. }
  81. func (h *keysHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
  82. if !allowMethod(w, r.Method, "HEAD", "GET", "PUT", "POST", "DELETE") {
  83. return
  84. }
  85. w.Header().Set("X-Etcd-Cluster-ID", h.clusterInfo.ID().String())
  86. ctx, cancel := context.WithTimeout(context.Background(), h.timeout)
  87. defer cancel()
  88. rr, err := parseKeyRequest(r, etcdserver.GenID(), clockwork.NewRealClock())
  89. if err != nil {
  90. writeError(w, err)
  91. return
  92. }
  93. resp, err := h.server.Do(ctx, rr)
  94. if err != nil {
  95. err = trimErrorPrefix(err, etcdserver.StoreKeysPrefix)
  96. writeError(w, err)
  97. return
  98. }
  99. switch {
  100. case resp.Event != nil:
  101. if err := writeKeyEvent(w, resp.Event, h.timer); err != nil {
  102. // Should never be reached
  103. log.Printf("error writing event: %v", err)
  104. }
  105. case resp.Watcher != nil:
  106. ctx, cancel := context.WithTimeout(context.Background(), defaultWatchTimeout)
  107. defer cancel()
  108. handleKeyWatch(ctx, w, resp.Watcher, rr.Stream, h.timer)
  109. default:
  110. writeError(w, errors.New("received response with no Event/Watcher!"))
  111. }
  112. }
  113. type deprecatedMachinesHandler struct {
  114. clusterInfo etcdserver.ClusterInfo
  115. }
  116. func (h *deprecatedMachinesHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
  117. if !allowMethod(w, r.Method, "GET", "HEAD") {
  118. return
  119. }
  120. endpoints := h.clusterInfo.ClientURLs()
  121. w.Write([]byte(strings.Join(endpoints, ", ")))
  122. }
  123. type membersHandler struct {
  124. server etcdserver.Server
  125. clusterInfo etcdserver.ClusterInfo
  126. clock clockwork.Clock
  127. }
  128. func (h *membersHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
  129. if !allowMethod(w, r.Method, "GET", "POST", "DELETE") {
  130. return
  131. }
  132. w.Header().Set("X-Etcd-Cluster-ID", h.clusterInfo.ID().String())
  133. ctx, cancel := context.WithTimeout(context.Background(), defaultServerTimeout)
  134. defer cancel()
  135. switch r.Method {
  136. case "GET":
  137. if trimPrefix(r.URL.Path, membersPrefix) != "" {
  138. writeError(w, httptypes.NewHTTPError(http.StatusNotFound, "Not found"))
  139. return
  140. }
  141. mc := newMemberCollection(h.clusterInfo.Members())
  142. w.Header().Set("Content-Type", "application/json")
  143. if err := json.NewEncoder(w).Encode(mc); err != nil {
  144. log.Printf("etcdhttp: %v", err)
  145. }
  146. case "POST":
  147. ctype := r.Header.Get("Content-Type")
  148. if ctype != "application/json" {
  149. writeError(w, httptypes.NewHTTPError(http.StatusUnsupportedMediaType, fmt.Sprintf("Bad Content-Type %s, accept application/json", ctype)))
  150. return
  151. }
  152. b, err := ioutil.ReadAll(r.Body)
  153. if err != nil {
  154. writeError(w, httptypes.NewHTTPError(http.StatusBadRequest, err.Error()))
  155. return
  156. }
  157. req := httptypes.MemberCreateRequest{}
  158. if err := json.Unmarshal(b, &req); err != nil {
  159. writeError(w, httptypes.NewHTTPError(http.StatusBadRequest, err.Error()))
  160. return
  161. }
  162. now := h.clock.Now()
  163. m := etcdserver.NewMember("", req.PeerURLs, "", &now)
  164. if err := h.server.AddMember(ctx, *m); err != nil {
  165. log.Printf("etcdhttp: error adding node %s: %v", m.ID, err)
  166. writeError(w, err)
  167. return
  168. }
  169. res := newMember(m)
  170. w.Header().Set("Content-Type", "application/json")
  171. w.WriteHeader(http.StatusCreated)
  172. if err := json.NewEncoder(w).Encode(res); err != nil {
  173. log.Printf("etcdhttp: %v", err)
  174. }
  175. case "DELETE":
  176. idStr := trimPrefix(r.URL.Path, membersPrefix)
  177. if idStr == "" {
  178. http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
  179. return
  180. }
  181. id, err := types.IDFromString(idStr)
  182. if err != nil {
  183. writeError(w, httptypes.NewHTTPError(http.StatusBadRequest, err.Error()))
  184. return
  185. }
  186. err = h.server.RemoveMember(ctx, uint64(id))
  187. switch {
  188. case err == etcdserver.ErrIDRemoved:
  189. writeError(w, httptypes.NewHTTPError(http.StatusGone, fmt.Sprintf("Member permanently removed: %s", idStr)))
  190. case err == etcdserver.ErrIDNotFound:
  191. writeError(w, httptypes.NewHTTPError(http.StatusNotFound, fmt.Sprintf("No such member: %s", idStr)))
  192. case err != nil:
  193. log.Printf("etcdhttp: error removing node %s: %v", id, err)
  194. writeError(w, err)
  195. default:
  196. w.WriteHeader(http.StatusNoContent)
  197. }
  198. }
  199. }
  200. type statsHandler struct {
  201. stats etcdserver.Stats
  202. }
  203. func (h *statsHandler) serveStore(w http.ResponseWriter, r *http.Request) {
  204. if !allowMethod(w, r.Method, "GET") {
  205. return
  206. }
  207. w.Header().Set("Content-Type", "application/json")
  208. w.Write(h.stats.StoreStats())
  209. }
  210. func (h *statsHandler) serveSelf(w http.ResponseWriter, r *http.Request) {
  211. if !allowMethod(w, r.Method, "GET") {
  212. return
  213. }
  214. w.Header().Set("Content-Type", "application/json")
  215. w.Write(h.stats.SelfStats())
  216. }
  217. func (h *statsHandler) serveLeader(w http.ResponseWriter, r *http.Request) {
  218. if !allowMethod(w, r.Method, "GET") {
  219. return
  220. }
  221. w.Header().Set("Content-Type", "application/json")
  222. w.Write(h.stats.LeaderStats())
  223. }
  224. func serveVersion(w http.ResponseWriter, r *http.Request) {
  225. if !allowMethod(w, r.Method, "GET") {
  226. return
  227. }
  228. w.Write([]byte("etcd " + version.Version))
  229. }
  230. // parseKeyRequest converts a received http.Request on keysPrefix to
  231. // a server Request, performing validation of supplied fields as appropriate.
  232. // If any validation fails, an empty Request and non-nil error is returned.
  233. func parseKeyRequest(r *http.Request, id uint64, clock clockwork.Clock) (etcdserverpb.Request, error) {
  234. emptyReq := etcdserverpb.Request{}
  235. err := r.ParseForm()
  236. if err != nil {
  237. return emptyReq, etcdErr.NewRequestError(
  238. etcdErr.EcodeInvalidForm,
  239. err.Error(),
  240. )
  241. }
  242. if !strings.HasPrefix(r.URL.Path, keysPrefix) {
  243. return emptyReq, etcdErr.NewRequestError(
  244. etcdErr.EcodeInvalidForm,
  245. "incorrect key prefix",
  246. )
  247. }
  248. p := path.Join(etcdserver.StoreKeysPrefix, r.URL.Path[len(keysPrefix):])
  249. var pIdx, wIdx uint64
  250. if pIdx, err = getUint64(r.Form, "prevIndex"); err != nil {
  251. return emptyReq, etcdErr.NewRequestError(
  252. etcdErr.EcodeIndexNaN,
  253. `invalid value for "prevIndex"`,
  254. )
  255. }
  256. if wIdx, err = getUint64(r.Form, "waitIndex"); err != nil {
  257. return emptyReq, etcdErr.NewRequestError(
  258. etcdErr.EcodeIndexNaN,
  259. `invalid value for "waitIndex"`,
  260. )
  261. }
  262. var rec, sort, wait, dir, quorum, stream bool
  263. if rec, err = getBool(r.Form, "recursive"); err != nil {
  264. return emptyReq, etcdErr.NewRequestError(
  265. etcdErr.EcodeInvalidField,
  266. `invalid value for "recursive"`,
  267. )
  268. }
  269. if sort, err = getBool(r.Form, "sorted"); err != nil {
  270. return emptyReq, etcdErr.NewRequestError(
  271. etcdErr.EcodeInvalidField,
  272. `invalid value for "sorted"`,
  273. )
  274. }
  275. if wait, err = getBool(r.Form, "wait"); err != nil {
  276. return emptyReq, etcdErr.NewRequestError(
  277. etcdErr.EcodeInvalidField,
  278. `invalid value for "wait"`,
  279. )
  280. }
  281. // TODO(jonboulle): define what parameters dir is/isn't compatible with?
  282. if dir, err = getBool(r.Form, "dir"); err != nil {
  283. return emptyReq, etcdErr.NewRequestError(
  284. etcdErr.EcodeInvalidField,
  285. `invalid value for "dir"`,
  286. )
  287. }
  288. if quorum, err = getBool(r.Form, "quorum"); err != nil {
  289. return emptyReq, etcdErr.NewRequestError(
  290. etcdErr.EcodeInvalidField,
  291. `invalid value for "quorum"`,
  292. )
  293. }
  294. if stream, err = getBool(r.Form, "stream"); err != nil {
  295. return emptyReq, etcdErr.NewRequestError(
  296. etcdErr.EcodeInvalidField,
  297. `invalid value for "stream"`,
  298. )
  299. }
  300. if wait && r.Method != "GET" {
  301. return emptyReq, etcdErr.NewRequestError(
  302. etcdErr.EcodeInvalidField,
  303. `"wait" can only be used with GET requests`,
  304. )
  305. }
  306. pV := r.FormValue("prevValue")
  307. if _, ok := r.Form["prevValue"]; ok && pV == "" {
  308. return emptyReq, etcdErr.NewRequestError(
  309. etcdErr.EcodePrevValueRequired,
  310. `"prevValue" cannot be empty`,
  311. )
  312. }
  313. // TTL is nullable, so leave it null if not specified
  314. // or an empty string
  315. var ttl *uint64
  316. if len(r.FormValue("ttl")) > 0 {
  317. i, err := getUint64(r.Form, "ttl")
  318. if err != nil {
  319. return emptyReq, etcdErr.NewRequestError(
  320. etcdErr.EcodeTTLNaN,
  321. `invalid value for "ttl"`,
  322. )
  323. }
  324. ttl = &i
  325. }
  326. // prevExist is nullable, so leave it null if not specified
  327. var pe *bool
  328. if _, ok := r.Form["prevExist"]; ok {
  329. bv, err := getBool(r.Form, "prevExist")
  330. if err != nil {
  331. return emptyReq, etcdErr.NewRequestError(
  332. etcdErr.EcodeInvalidField,
  333. "invalid value for prevExist",
  334. )
  335. }
  336. pe = &bv
  337. }
  338. rr := etcdserverpb.Request{
  339. ID: id,
  340. Method: r.Method,
  341. Path: p,
  342. Val: r.FormValue("value"),
  343. Dir: dir,
  344. PrevValue: pV,
  345. PrevIndex: pIdx,
  346. PrevExist: pe,
  347. Wait: wait,
  348. Since: wIdx,
  349. Recursive: rec,
  350. Sorted: sort,
  351. Quorum: quorum,
  352. Stream: stream,
  353. }
  354. if pe != nil {
  355. rr.PrevExist = pe
  356. }
  357. // Null TTL is equivalent to unset Expiration
  358. if ttl != nil {
  359. expr := time.Duration(*ttl) * time.Second
  360. rr.Expiration = clock.Now().Add(expr).UnixNano()
  361. }
  362. return rr, nil
  363. }
  364. // writeKeyEvent trims the prefix of key path in a single Event under
  365. // StoreKeysPrefix, serializes it and writes the resulting JSON to the given
  366. // ResponseWriter, along with the appropriate headers.
  367. func writeKeyEvent(w http.ResponseWriter, ev *store.Event, rt etcdserver.RaftTimer) error {
  368. if ev == nil {
  369. return errors.New("cannot write empty Event!")
  370. }
  371. w.Header().Set("Content-Type", "application/json")
  372. w.Header().Set("X-Etcd-Index", fmt.Sprint(ev.EtcdIndex))
  373. w.Header().Set("X-Raft-Index", fmt.Sprint(rt.Index()))
  374. w.Header().Set("X-Raft-Term", fmt.Sprint(rt.Term()))
  375. if ev.IsCreated() {
  376. w.WriteHeader(http.StatusCreated)
  377. }
  378. ev = trimEventPrefix(ev, etcdserver.StoreKeysPrefix)
  379. return json.NewEncoder(w).Encode(ev)
  380. }
  381. func handleKeyWatch(ctx context.Context, w http.ResponseWriter, wa store.Watcher, stream bool, rt etcdserver.RaftTimer) {
  382. defer wa.Remove()
  383. ech := wa.EventChan()
  384. var nch <-chan bool
  385. if x, ok := w.(http.CloseNotifier); ok {
  386. nch = x.CloseNotify()
  387. }
  388. w.Header().Set("Content-Type", "application/json")
  389. w.Header().Set("X-Etcd-Index", fmt.Sprint(wa.StartIndex()))
  390. w.Header().Set("X-Raft-Index", fmt.Sprint(rt.Index()))
  391. w.Header().Set("X-Raft-Term", fmt.Sprint(rt.Term()))
  392. w.WriteHeader(http.StatusOK)
  393. // Ensure headers are flushed early, in case of long polling
  394. w.(http.Flusher).Flush()
  395. for {
  396. select {
  397. case <-nch:
  398. // Client closed connection. Nothing to do.
  399. return
  400. case <-ctx.Done():
  401. // Timed out. net/http will close the connection for us, so nothing to do.
  402. return
  403. case ev, ok := <-ech:
  404. if !ok {
  405. // If the channel is closed this may be an indication of
  406. // that notifications are much more than we are able to
  407. // send to the client in time. Then we simply end streaming.
  408. return
  409. }
  410. ev = trimEventPrefix(ev, etcdserver.StoreKeysPrefix)
  411. if err := json.NewEncoder(w).Encode(ev); err != nil {
  412. // Should never be reached
  413. log.Printf("error writing event: %v\n", err)
  414. return
  415. }
  416. if !stream {
  417. return
  418. }
  419. w.(http.Flusher).Flush()
  420. }
  421. }
  422. }
  423. func trimEventPrefix(ev *store.Event, prefix string) *store.Event {
  424. if ev == nil {
  425. return nil
  426. }
  427. // Since the *Event may reference one in the store history
  428. // history, we must copy it before modifying
  429. e := ev.Clone()
  430. e.Node = trimNodeExternPrefix(e.Node, prefix)
  431. e.PrevNode = trimNodeExternPrefix(e.PrevNode, prefix)
  432. return e
  433. }
  434. func trimNodeExternPrefix(n *store.NodeExtern, prefix string) *store.NodeExtern {
  435. if n == nil {
  436. return nil
  437. }
  438. n.Key = strings.TrimPrefix(n.Key, prefix)
  439. for _, nn := range n.Nodes {
  440. nn = trimNodeExternPrefix(nn, prefix)
  441. }
  442. return n
  443. }
  444. func trimErrorPrefix(err error, prefix string) error {
  445. if e, ok := err.(*etcdErr.Error); ok {
  446. e.Cause = strings.TrimPrefix(e.Cause, prefix)
  447. }
  448. return err
  449. }
  450. // getUint64 extracts a uint64 by the given key from a Form. If the key does
  451. // not exist in the form, 0 is returned. If the key exists but the value is
  452. // badly formed, an error is returned. If multiple values are present only the
  453. // first is considered.
  454. func getUint64(form url.Values, key string) (i uint64, err error) {
  455. if vals, ok := form[key]; ok {
  456. i, err = strconv.ParseUint(vals[0], 10, 64)
  457. }
  458. return
  459. }
  460. // getBool extracts a bool by the given key from a Form. If the key does not
  461. // exist in the form, false is returned. If the key exists but the value is
  462. // badly formed, an error is returned. If multiple values are present only the
  463. // first is considered.
  464. func getBool(form url.Values, key string) (b bool, err error) {
  465. if vals, ok := form[key]; ok {
  466. b, err = strconv.ParseBool(vals[0])
  467. }
  468. return
  469. }
  470. // trimPrefix removes a given prefix and any slash following the prefix
  471. // e.g.: trimPrefix("foo", "foo") == trimPrefix("foo/", "foo") == ""
  472. func trimPrefix(p, prefix string) (s string) {
  473. s = strings.TrimPrefix(p, prefix)
  474. s = strings.TrimPrefix(s, "/")
  475. return
  476. }
  477. func newMemberCollection(ms []*etcdserver.Member) *httptypes.MemberCollection {
  478. c := httptypes.MemberCollection(make([]httptypes.Member, len(ms)))
  479. for i, m := range ms {
  480. c[i] = newMember(m)
  481. }
  482. return &c
  483. }
  484. func newMember(m *etcdserver.Member) httptypes.Member {
  485. tm := httptypes.Member{
  486. ID: m.ID.String(),
  487. Name: m.Name,
  488. PeerURLs: make([]string, len(m.PeerURLs)),
  489. ClientURLs: make([]string, len(m.ClientURLs)),
  490. }
  491. copy(tm.PeerURLs, m.PeerURLs)
  492. copy(tm.ClientURLs, m.ClientURLs)
  493. return tm
  494. }