http.go 17 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590
  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. "sort"
  24. "strconv"
  25. "strings"
  26. "time"
  27. "github.com/coreos/etcd/Godeps/_workspace/src/code.google.com/p/go.net/context"
  28. "github.com/coreos/etcd/Godeps/_workspace/src/github.com/jonboulle/clockwork"
  29. etcdErr "github.com/coreos/etcd/error"
  30. "github.com/coreos/etcd/etcdserver"
  31. "github.com/coreos/etcd/etcdserver/etcdserverpb"
  32. "github.com/coreos/etcd/pkg/types"
  33. "github.com/coreos/etcd/raft/raftpb"
  34. "github.com/coreos/etcd/store"
  35. )
  36. const (
  37. keysPrefix = "/v2/keys"
  38. deprecatedMachinesPrefix = "/v2/machines"
  39. adminMembersPrefix = "/v2/admin/members/"
  40. raftPrefix = "/raft"
  41. statsPrefix = "/v2/stats"
  42. // time to wait for response from EtcdServer requests
  43. defaultServerTimeout = 500 * time.Millisecond
  44. // time to wait for a Watch request
  45. defaultWatchTimeout = 5 * time.Minute
  46. )
  47. var errClosed = errors.New("etcdhttp: client closed connection")
  48. // NewClientHandler generates a muxed http.Handler with the given parameters to serve etcd client requests.
  49. func NewClientHandler(server *etcdserver.EtcdServer) http.Handler {
  50. sh := &serverHandler{
  51. server: server,
  52. clusterInfo: server.Cluster,
  53. stats: server,
  54. timer: server,
  55. timeout: defaultServerTimeout,
  56. clock: clockwork.NewRealClock(),
  57. }
  58. mux := http.NewServeMux()
  59. mux.HandleFunc(keysPrefix, sh.serveKeys)
  60. mux.HandleFunc(keysPrefix+"/", sh.serveKeys)
  61. mux.HandleFunc(statsPrefix+"/store", sh.serveStoreStats)
  62. mux.HandleFunc(statsPrefix+"/self", sh.serveSelfStats)
  63. mux.HandleFunc(statsPrefix+"/leader", sh.serveLeaderStats)
  64. // TODO: dynamic configuration may make this outdated. take care of it.
  65. // TODO: dynamic configuration may introduce race also.
  66. // TODO: add serveMembers
  67. mux.HandleFunc(deprecatedMachinesPrefix, sh.serveMachines)
  68. mux.HandleFunc(adminMembersPrefix, sh.serveAdminMembers)
  69. mux.HandleFunc("/", http.NotFound)
  70. return mux
  71. }
  72. // NewPeerHandler generates an http.Handler to handle etcd peer (raft) requests.
  73. func NewPeerHandler(server *etcdserver.EtcdServer) http.Handler {
  74. sh := &serverHandler{
  75. server: server,
  76. stats: server,
  77. clusterInfo: server.Cluster,
  78. clock: clockwork.NewRealClock(),
  79. }
  80. mux := http.NewServeMux()
  81. mux.HandleFunc(raftPrefix, sh.serveRaft)
  82. mux.HandleFunc("/", http.NotFound)
  83. return mux
  84. }
  85. // serverHandler provides http.Handlers for etcd client and raft communication.
  86. type serverHandler struct {
  87. timeout time.Duration
  88. server etcdserver.Server
  89. stats etcdserver.Stats
  90. timer etcdserver.RaftTimer
  91. clusterInfo etcdserver.ClusterInfo
  92. clock clockwork.Clock
  93. }
  94. func (h serverHandler) serveKeys(w http.ResponseWriter, r *http.Request) {
  95. if !allowMethod(w, r.Method, "GET", "PUT", "POST", "DELETE") {
  96. return
  97. }
  98. ctx, cancel := context.WithTimeout(context.Background(), h.timeout)
  99. defer cancel()
  100. rr, err := parseKeyRequest(r, etcdserver.GenID(), clockwork.NewRealClock())
  101. if err != nil {
  102. writeError(w, err)
  103. return
  104. }
  105. resp, err := h.server.Do(ctx, rr)
  106. if err != nil {
  107. writeError(w, err)
  108. return
  109. }
  110. switch {
  111. case resp.Event != nil:
  112. if err := writeKeyEvent(w, resp.Event, h.timer); err != nil {
  113. // Should never be reached
  114. log.Printf("error writing event: %v", err)
  115. }
  116. case resp.Watcher != nil:
  117. ctx, cancel := context.WithTimeout(context.Background(), defaultWatchTimeout)
  118. defer cancel()
  119. handleKeyWatch(ctx, w, resp.Watcher, rr.Stream, h.timer)
  120. default:
  121. writeError(w, errors.New("received response with no Event/Watcher!"))
  122. }
  123. }
  124. // serveMachines responds address list in the format '0.0.0.0, 1.1.1.1'.
  125. func (h serverHandler) serveMachines(w http.ResponseWriter, r *http.Request) {
  126. if !allowMethod(w, r.Method, "GET", "HEAD") {
  127. return
  128. }
  129. endpoints := h.clusterInfo.ClientURLs()
  130. w.Write([]byte(strings.Join(endpoints, ", ")))
  131. }
  132. func (h serverHandler) serveAdminMembers(w http.ResponseWriter, r *http.Request) {
  133. if !allowMethod(w, r.Method, "GET", "POST", "DELETE") {
  134. return
  135. }
  136. ctx, cancel := context.WithTimeout(context.Background(), defaultServerTimeout)
  137. defer cancel()
  138. switch r.Method {
  139. case "GET":
  140. idStr := strings.TrimPrefix(r.URL.Path, adminMembersPrefix)
  141. if idStr == "" {
  142. msmap := h.clusterStore.Get().Members()
  143. ms := make(SortableMemberSlice, 0, len(msmap))
  144. for _, m := range msmap {
  145. ms = append(ms, m)
  146. }
  147. sort.Sort(ms)
  148. w.Header().Set("Content-Type", "application/json")
  149. if err := json.NewEncoder(w).Encode(ms); err != nil {
  150. log.Printf("etcdhttp: %v", err)
  151. }
  152. return
  153. }
  154. id, err := strconv.ParseUint(idStr, 16, 64)
  155. if err != nil {
  156. http.Error(w, err.Error(), http.StatusBadRequest)
  157. return
  158. }
  159. m := h.clusterStore.Get().FindID(id)
  160. if m == nil {
  161. http.Error(w, "member not found", http.StatusNotFound)
  162. return
  163. }
  164. w.Header().Set("Content-Type", "application/json")
  165. if err := json.NewEncoder(w).Encode(m); err != nil {
  166. log.Printf("etcdhttp: %v", err)
  167. }
  168. return
  169. case "POST":
  170. ctype := r.Header.Get("Content-Type")
  171. if ctype != "application/json" {
  172. http.Error(w, fmt.Sprintf("bad Content-Type %s, accept application/json", ctype), http.StatusBadRequest)
  173. return
  174. }
  175. b, err := ioutil.ReadAll(r.Body)
  176. if err != nil {
  177. http.Error(w, err.Error(), http.StatusBadRequest)
  178. return
  179. }
  180. raftAttr := etcdserver.RaftAttributes{}
  181. if err := json.Unmarshal(b, &raftAttr); err != nil {
  182. http.Error(w, err.Error(), http.StatusBadRequest)
  183. return
  184. }
  185. validURLs, err := types.NewURLs(raftAttr.PeerURLs)
  186. if err != nil {
  187. http.Error(w, "bad peer urls", http.StatusBadRequest)
  188. return
  189. }
  190. now := h.clock.Now()
  191. m := etcdserver.NewMember("", validURLs, "", &now)
  192. if err := h.server.AddMember(ctx, *m); err != nil {
  193. log.Printf("etcdhttp: error adding node %x: %v", m.ID, err)
  194. writeError(w, err)
  195. return
  196. }
  197. log.Printf("etcdhttp: added node %x with peer urls %v", m.ID, raftAttr.PeerURLs)
  198. w.Header().Set("Content-Type", "application/json")
  199. w.WriteHeader(http.StatusCreated)
  200. if err := json.NewEncoder(w).Encode(m); err != nil {
  201. log.Printf("etcdhttp: %v", err)
  202. }
  203. case "DELETE":
  204. idStr := strings.TrimPrefix(r.URL.Path, adminMembersPrefix)
  205. id, err := strconv.ParseUint(idStr, 16, 64)
  206. if err != nil {
  207. http.Error(w, err.Error(), http.StatusBadRequest)
  208. return
  209. }
  210. log.Printf("etcdhttp: remove node %x", id)
  211. if err := h.server.RemoveMember(ctx, id); err != nil {
  212. log.Printf("etcdhttp: error removing node %x: %v", id, err)
  213. writeError(w, err)
  214. return
  215. }
  216. w.WriteHeader(http.StatusNoContent)
  217. }
  218. }
  219. func (h serverHandler) serveStoreStats(w http.ResponseWriter, r *http.Request) {
  220. if !allowMethod(w, r.Method, "GET") {
  221. return
  222. }
  223. w.Header().Set("Content-Type", "application/json")
  224. w.Write(h.stats.StoreStats())
  225. }
  226. func (h serverHandler) serveSelfStats(w http.ResponseWriter, r *http.Request) {
  227. if !allowMethod(w, r.Method, "GET") {
  228. return
  229. }
  230. w.Header().Set("Content-Type", "application/json")
  231. w.Write(h.stats.SelfStats())
  232. }
  233. func (h serverHandler) serveLeaderStats(w http.ResponseWriter, r *http.Request) {
  234. if !allowMethod(w, r.Method, "GET") {
  235. return
  236. }
  237. w.Header().Set("Content-Type", "application/json")
  238. w.Write(h.stats.LeaderStats())
  239. }
  240. func (h serverHandler) serveRaft(w http.ResponseWriter, r *http.Request) {
  241. if !allowMethod(w, r.Method, "POST") {
  242. return
  243. }
  244. wcid := strconv.FormatUint(h.clusterInfo.ID(), 16)
  245. w.Header().Set("X-Etcd-Cluster-ID", wcid)
  246. gcid := r.Header.Get("X-Etcd-Cluster-ID")
  247. if gcid != wcid {
  248. log.Printf("etcdhttp: request ignored due to cluster ID mismatch got %s want %s", gcid, wcid)
  249. http.Error(w, "clusterID mismatch", http.StatusPreconditionFailed)
  250. return
  251. }
  252. b, err := ioutil.ReadAll(r.Body)
  253. if err != nil {
  254. log.Println("etcdhttp: error reading raft message:", err)
  255. http.Error(w, "error reading raft message", http.StatusBadRequest)
  256. return
  257. }
  258. var m raftpb.Message
  259. if err := m.Unmarshal(b); err != nil {
  260. log.Println("etcdhttp: error unmarshaling raft message:", err)
  261. http.Error(w, "error unmarshaling raft message", http.StatusBadRequest)
  262. return
  263. }
  264. log.Printf("etcdhttp: raft recv message from %x: %+v", m.From, m)
  265. if err := h.server.Process(context.TODO(), m); err != nil {
  266. log.Println("etcdhttp: error processing raft message:", err)
  267. switch err {
  268. case etcdserver.ErrRemoved:
  269. http.Error(w, "cannot process message from removed node", http.StatusForbidden)
  270. default:
  271. writeError(w, err)
  272. }
  273. return
  274. }
  275. if m.Type == raftpb.MsgApp {
  276. h.stats.UpdateRecvApp(m.From, r.ContentLength)
  277. }
  278. w.WriteHeader(http.StatusNoContent)
  279. }
  280. // parseKeyRequest converts a received http.Request on keysPrefix to
  281. // a server Request, performing validation of supplied fields as appropriate.
  282. // If any validation fails, an empty Request and non-nil error is returned.
  283. func parseKeyRequest(r *http.Request, id uint64, clock clockwork.Clock) (etcdserverpb.Request, error) {
  284. emptyReq := etcdserverpb.Request{}
  285. err := r.ParseForm()
  286. if err != nil {
  287. return emptyReq, etcdErr.NewRequestError(
  288. etcdErr.EcodeInvalidForm,
  289. err.Error(),
  290. )
  291. }
  292. if !strings.HasPrefix(r.URL.Path, keysPrefix) {
  293. return emptyReq, etcdErr.NewRequestError(
  294. etcdErr.EcodeInvalidForm,
  295. "incorrect key prefix",
  296. )
  297. }
  298. p := path.Join(etcdserver.StoreKeysPrefix, r.URL.Path[len(keysPrefix):])
  299. var pIdx, wIdx uint64
  300. if pIdx, err = getUint64(r.Form, "prevIndex"); err != nil {
  301. return emptyReq, etcdErr.NewRequestError(
  302. etcdErr.EcodeIndexNaN,
  303. `invalid value for "prevIndex"`,
  304. )
  305. }
  306. if wIdx, err = getUint64(r.Form, "waitIndex"); err != nil {
  307. return emptyReq, etcdErr.NewRequestError(
  308. etcdErr.EcodeIndexNaN,
  309. `invalid value for "waitIndex"`,
  310. )
  311. }
  312. var rec, sort, wait, dir, stream bool
  313. if rec, err = getBool(r.Form, "recursive"); err != nil {
  314. return emptyReq, etcdErr.NewRequestError(
  315. etcdErr.EcodeInvalidField,
  316. `invalid value for "recursive"`,
  317. )
  318. }
  319. if sort, err = getBool(r.Form, "sorted"); err != nil {
  320. return emptyReq, etcdErr.NewRequestError(
  321. etcdErr.EcodeInvalidField,
  322. `invalid value for "sorted"`,
  323. )
  324. }
  325. if wait, err = getBool(r.Form, "wait"); err != nil {
  326. return emptyReq, etcdErr.NewRequestError(
  327. etcdErr.EcodeInvalidField,
  328. `invalid value for "wait"`,
  329. )
  330. }
  331. // TODO(jonboulle): define what parameters dir is/isn't compatible with?
  332. if dir, err = getBool(r.Form, "dir"); err != nil {
  333. return emptyReq, etcdErr.NewRequestError(
  334. etcdErr.EcodeInvalidField,
  335. `invalid value for "dir"`,
  336. )
  337. }
  338. if stream, err = getBool(r.Form, "stream"); err != nil {
  339. return emptyReq, etcdErr.NewRequestError(
  340. etcdErr.EcodeInvalidField,
  341. `invalid value for "stream"`,
  342. )
  343. }
  344. if wait && r.Method != "GET" {
  345. return emptyReq, etcdErr.NewRequestError(
  346. etcdErr.EcodeInvalidField,
  347. `"wait" can only be used with GET requests`,
  348. )
  349. }
  350. pV := r.FormValue("prevValue")
  351. if _, ok := r.Form["prevValue"]; ok && pV == "" {
  352. return emptyReq, etcdErr.NewRequestError(
  353. etcdErr.EcodeInvalidField,
  354. `"prevValue" cannot be empty`,
  355. )
  356. }
  357. // TTL is nullable, so leave it null if not specified
  358. // or an empty string
  359. var ttl *uint64
  360. if len(r.FormValue("ttl")) > 0 {
  361. i, err := getUint64(r.Form, "ttl")
  362. if err != nil {
  363. return emptyReq, etcdErr.NewRequestError(
  364. etcdErr.EcodeTTLNaN,
  365. `invalid value for "ttl"`,
  366. )
  367. }
  368. ttl = &i
  369. }
  370. // prevExist is nullable, so leave it null if not specified
  371. var pe *bool
  372. if _, ok := r.Form["prevExist"]; ok {
  373. bv, err := getBool(r.Form, "prevExist")
  374. if err != nil {
  375. return emptyReq, etcdErr.NewRequestError(
  376. etcdErr.EcodeInvalidField,
  377. "invalid value for prevExist",
  378. )
  379. }
  380. pe = &bv
  381. }
  382. rr := etcdserverpb.Request{
  383. ID: id,
  384. Method: r.Method,
  385. Path: p,
  386. Val: r.FormValue("value"),
  387. Dir: dir,
  388. PrevValue: pV,
  389. PrevIndex: pIdx,
  390. PrevExist: pe,
  391. Recursive: rec,
  392. Since: wIdx,
  393. Sorted: sort,
  394. Stream: stream,
  395. Wait: wait,
  396. }
  397. if pe != nil {
  398. rr.PrevExist = pe
  399. }
  400. // Null TTL is equivalent to unset Expiration
  401. if ttl != nil {
  402. expr := time.Duration(*ttl) * time.Second
  403. rr.Expiration = clock.Now().Add(expr).UnixNano()
  404. }
  405. return rr, nil
  406. }
  407. // getUint64 extracts a uint64 by the given key from a Form. If the key does
  408. // not exist in the form, 0 is returned. If the key exists but the value is
  409. // badly formed, an error is returned. If multiple values are present only the
  410. // first is considered.
  411. func getUint64(form url.Values, key string) (i uint64, err error) {
  412. if vals, ok := form[key]; ok {
  413. i, err = strconv.ParseUint(vals[0], 10, 64)
  414. }
  415. return
  416. }
  417. // getBool extracts a bool by the given key from a Form. If the key does not
  418. // exist in the form, false is returned. If the key exists but the value is
  419. // badly formed, an error is returned. If multiple values are present only the
  420. // first is considered.
  421. func getBool(form url.Values, key string) (b bool, err error) {
  422. if vals, ok := form[key]; ok {
  423. b, err = strconv.ParseBool(vals[0])
  424. }
  425. return
  426. }
  427. // writeError logs and writes the given Error to the ResponseWriter
  428. // If Error is an etcdErr, it is rendered to the ResponseWriter
  429. // Otherwise, it is assumed to be an InternalServerError
  430. func writeError(w http.ResponseWriter, err error) {
  431. if err == nil {
  432. return
  433. }
  434. log.Println(err)
  435. if e, ok := err.(*etcdErr.Error); ok {
  436. e.Write(w)
  437. } else {
  438. http.Error(w, "Internal Server Error", http.StatusInternalServerError)
  439. }
  440. }
  441. // writeKeyEvent trims the prefix of key path in a single Event under
  442. // StoreKeysPrefix, serializes it and writes the resulting JSON to the given
  443. // ResponseWriter, along with the appropriate headers.
  444. func writeKeyEvent(w http.ResponseWriter, ev *store.Event, rt etcdserver.RaftTimer) error {
  445. if ev == nil {
  446. return errors.New("cannot write empty Event!")
  447. }
  448. w.Header().Set("Content-Type", "application/json")
  449. w.Header().Set("X-Etcd-Index", fmt.Sprint(ev.EtcdIndex))
  450. w.Header().Set("X-Raft-Index", fmt.Sprint(rt.Index()))
  451. w.Header().Set("X-Raft-Term", fmt.Sprint(rt.Term()))
  452. if ev.IsCreated() {
  453. w.WriteHeader(http.StatusCreated)
  454. }
  455. ev = trimEventPrefix(ev, etcdserver.StoreKeysPrefix)
  456. return json.NewEncoder(w).Encode(ev)
  457. }
  458. func handleKeyWatch(ctx context.Context, w http.ResponseWriter, wa store.Watcher, stream bool, rt etcdserver.RaftTimer) {
  459. defer wa.Remove()
  460. ech := wa.EventChan()
  461. var nch <-chan bool
  462. if x, ok := w.(http.CloseNotifier); ok {
  463. nch = x.CloseNotify()
  464. }
  465. w.Header().Set("Content-Type", "application/json")
  466. w.Header().Set("X-Etcd-Index", fmt.Sprint(wa.StartIndex()))
  467. w.Header().Set("X-Raft-Index", fmt.Sprint(rt.Index()))
  468. w.Header().Set("X-Raft-Term", fmt.Sprint(rt.Term()))
  469. w.WriteHeader(http.StatusOK)
  470. // Ensure headers are flushed early, in case of long polling
  471. w.(http.Flusher).Flush()
  472. for {
  473. select {
  474. case <-nch:
  475. // Client closed connection. Nothing to do.
  476. return
  477. case <-ctx.Done():
  478. // Timed out. net/http will close the connection for us, so nothing to do.
  479. return
  480. case ev, ok := <-ech:
  481. if !ok {
  482. // If the channel is closed this may be an indication of
  483. // that notifications are much more than we are able to
  484. // send to the client in time. Then we simply end streaming.
  485. return
  486. }
  487. ev = trimEventPrefix(ev, etcdserver.StoreKeysPrefix)
  488. if err := json.NewEncoder(w).Encode(ev); err != nil {
  489. // Should never be reached
  490. log.Printf("error writing event: %v\n", err)
  491. return
  492. }
  493. if !stream {
  494. return
  495. }
  496. w.(http.Flusher).Flush()
  497. }
  498. }
  499. }
  500. // allowMethod verifies that the given method is one of the allowed methods,
  501. // and if not, it writes an error to w. A boolean is returned indicating
  502. // whether or not the method is allowed.
  503. func allowMethod(w http.ResponseWriter, m string, ms ...string) bool {
  504. for _, meth := range ms {
  505. if m == meth {
  506. return true
  507. }
  508. }
  509. w.Header().Set("Allow", strings.Join(ms, ","))
  510. http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
  511. return false
  512. }
  513. func trimEventPrefix(ev *store.Event, prefix string) *store.Event {
  514. if ev == nil {
  515. return nil
  516. }
  517. ev.Node = trimNodeExternPrefix(ev.Node, prefix)
  518. ev.PrevNode = trimNodeExternPrefix(ev.PrevNode, prefix)
  519. return ev
  520. }
  521. func trimNodeExternPrefix(n *store.NodeExtern, prefix string) *store.NodeExtern {
  522. if n == nil {
  523. return nil
  524. }
  525. n.Key = strings.TrimPrefix(n.Key, prefix)
  526. for _, nn := range n.Nodes {
  527. nn = trimNodeExternPrefix(nn, prefix)
  528. }
  529. return n
  530. }
  531. type SortableMemberSlice []*etcdserver.Member
  532. func (s SortableMemberSlice) Len() int { return len(s) }
  533. func (s SortableMemberSlice) Less(i, j int) bool { return s[i].ID < s[j].ID }
  534. func (s SortableMemberSlice) Swap(i, j int) { s[i], s[j] = s[j], s[i] }