client.go 15 KB

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