client.go 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512
  1. // Copyright 2015 CoreOS, Inc.
  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 client
  15. import (
  16. "errors"
  17. "fmt"
  18. "io/ioutil"
  19. "math/rand"
  20. "net"
  21. "net/http"
  22. "net/url"
  23. "reflect"
  24. "sort"
  25. "sync"
  26. "time"
  27. "github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
  28. )
  29. var (
  30. ErrNoEndpoints = errors.New("client: no endpoints available")
  31. ErrTooManyRedirects = errors.New("client: too many redirects")
  32. ErrClusterUnavailable = errors.New("client: etcd cluster is unavailable or misconfigured")
  33. errTooManyRedirectChecks = errors.New("client: too many redirect checks")
  34. )
  35. var DefaultRequestTimeout = 5 * time.Second
  36. var DefaultTransport CancelableTransport = &http.Transport{
  37. Proxy: http.ProxyFromEnvironment,
  38. Dial: (&net.Dialer{
  39. Timeout: 30 * time.Second,
  40. KeepAlive: 30 * time.Second,
  41. }).Dial,
  42. TLSHandshakeTimeout: 10 * time.Second,
  43. }
  44. type Config struct {
  45. // Endpoints defines a set of URLs (schemes, hosts and ports only)
  46. // that can be used to communicate with a logical etcd cluster. For
  47. // example, a three-node cluster could be provided like so:
  48. //
  49. // Endpoints: []string{
  50. // "http://node1.example.com:2379",
  51. // "http://node2.example.com:2379",
  52. // "http://node3.example.com:2379",
  53. // }
  54. //
  55. // If multiple endpoints are provided, the Client will attempt to
  56. // use them all in the event that one or more of them are unusable.
  57. //
  58. // If Client.Sync is ever called, the Client may cache an alternate
  59. // set of endpoints to continue operation.
  60. Endpoints []string
  61. // Transport is used by the Client to drive HTTP requests. If not
  62. // provided, DefaultTransport will be used.
  63. Transport CancelableTransport
  64. // CheckRedirect specifies the policy for handling HTTP redirects.
  65. // If CheckRedirect is not nil, the Client calls it before
  66. // following an HTTP redirect. The sole argument is the number of
  67. // requests that have alrady been made. If CheckRedirect returns
  68. // an error, Client.Do will not make any further requests and return
  69. // the error back it to the caller.
  70. //
  71. // If CheckRedirect is nil, the Client uses its default policy,
  72. // which is to stop after 10 consecutive requests.
  73. CheckRedirect CheckRedirectFunc
  74. // Username specifies the user credential to add as an authorization header
  75. Username string
  76. // Password is the password for the specified user to add as an authorization header
  77. // to the request.
  78. Password string
  79. // HeaderTimeoutPerRequest specifies the time limit to wait for response
  80. // header in a single request made by the Client. The timeout includes
  81. // connection time, any redirects, and header wait time.
  82. //
  83. // For non-watch GET request, server returns the response body immediately.
  84. // For PUT/POST/DELETE request, server will attempt to commit request
  85. // before responding, which is expected to take `100ms + 2 * RTT`.
  86. // For watch request, server returns the header immediately to notify Client
  87. // watch start. But if server is behind some kind of proxy, the response
  88. // header may be cached at proxy, and Client cannot rely on this behavior.
  89. //
  90. // One API call may send multiple requests to different etcd servers until it
  91. // succeeds. Use context of the API to specify the overall timeout.
  92. //
  93. // A HeaderTimeoutPerRequest of zero means no timeout.
  94. HeaderTimeoutPerRequest time.Duration
  95. }
  96. func (cfg *Config) transport() CancelableTransport {
  97. if cfg.Transport == nil {
  98. return DefaultTransport
  99. }
  100. return cfg.Transport
  101. }
  102. func (cfg *Config) checkRedirect() CheckRedirectFunc {
  103. if cfg.CheckRedirect == nil {
  104. return DefaultCheckRedirect
  105. }
  106. return cfg.CheckRedirect
  107. }
  108. // CancelableTransport mimics net/http.Transport, but requires that
  109. // the object also support request cancellation.
  110. type CancelableTransport interface {
  111. http.RoundTripper
  112. CancelRequest(req *http.Request)
  113. }
  114. type CheckRedirectFunc func(via int) error
  115. // DefaultCheckRedirect follows up to 10 redirects, but no more.
  116. var DefaultCheckRedirect CheckRedirectFunc = func(via int) error {
  117. if via > 10 {
  118. return ErrTooManyRedirects
  119. }
  120. return nil
  121. }
  122. type Client interface {
  123. // Sync updates the internal cache of the etcd cluster's membership.
  124. Sync(context.Context) error
  125. // AutoSync periodically calls Sync() every given interval.
  126. // The recommended sync interval is 10 seconds to 1 minute, which does
  127. // not bring too much overhead to server and makes client catch up the
  128. // cluster change in time.
  129. //
  130. // The example to use it:
  131. //
  132. // for {
  133. // err := client.AutoSync(ctx, 10*time.Second)
  134. // if err == context.DeadlineExceeded || err == context.Canceled {
  135. // break
  136. // }
  137. // log.Print(err)
  138. // }
  139. AutoSync(context.Context, time.Duration) error
  140. // Endpoints returns a copy of the current set of API endpoints used
  141. // by Client to resolve HTTP requests. If Sync has ever been called,
  142. // this may differ from the initial Endpoints provided in the Config.
  143. Endpoints() []string
  144. httpClient
  145. }
  146. func New(cfg Config) (Client, error) {
  147. c := &httpClusterClient{
  148. clientFactory: newHTTPClientFactory(cfg.transport(), cfg.checkRedirect(), cfg.HeaderTimeoutPerRequest),
  149. rand: rand.New(rand.NewSource(int64(time.Now().Nanosecond()))),
  150. }
  151. if cfg.Username != "" {
  152. c.credentials = &credentials{
  153. username: cfg.Username,
  154. password: cfg.Password,
  155. }
  156. }
  157. if err := c.reset(cfg.Endpoints); err != nil {
  158. return nil, err
  159. }
  160. return c, nil
  161. }
  162. type httpClient interface {
  163. Do(context.Context, httpAction) (*http.Response, []byte, error)
  164. }
  165. func newHTTPClientFactory(tr CancelableTransport, cr CheckRedirectFunc, headerTimeout time.Duration) httpClientFactory {
  166. return func(ep url.URL) httpClient {
  167. return &redirectFollowingHTTPClient{
  168. checkRedirect: cr,
  169. client: &simpleHTTPClient{
  170. transport: tr,
  171. endpoint: ep,
  172. headerTimeout: headerTimeout,
  173. },
  174. }
  175. }
  176. }
  177. type credentials struct {
  178. username string
  179. password string
  180. }
  181. type httpClientFactory func(url.URL) httpClient
  182. type httpAction interface {
  183. HTTPRequest(url.URL) *http.Request
  184. }
  185. type httpClusterClient struct {
  186. clientFactory httpClientFactory
  187. endpoints []url.URL
  188. pinned int
  189. credentials *credentials
  190. sync.RWMutex
  191. rand *rand.Rand
  192. }
  193. func (c *httpClusterClient) reset(eps []string) error {
  194. if len(eps) == 0 {
  195. return ErrNoEndpoints
  196. }
  197. neps := make([]url.URL, len(eps))
  198. for i, ep := range eps {
  199. u, err := url.Parse(ep)
  200. if err != nil {
  201. return err
  202. }
  203. neps[i] = *u
  204. }
  205. c.endpoints = shuffleEndpoints(c.rand, neps)
  206. // TODO: pin old endpoint if possible, and rebalance when new endpoint appears
  207. c.pinned = 0
  208. return nil
  209. }
  210. func (c *httpClusterClient) Do(ctx context.Context, act httpAction) (*http.Response, []byte, error) {
  211. action := act
  212. c.RLock()
  213. leps := len(c.endpoints)
  214. eps := make([]url.URL, leps)
  215. n := copy(eps, c.endpoints)
  216. pinned := c.pinned
  217. if c.credentials != nil {
  218. action = &authedAction{
  219. act: act,
  220. credentials: *c.credentials,
  221. }
  222. }
  223. c.RUnlock()
  224. if leps == 0 {
  225. return nil, nil, ErrNoEndpoints
  226. }
  227. if leps != n {
  228. return nil, nil, errors.New("unable to pick endpoint: copy failed")
  229. }
  230. var resp *http.Response
  231. var body []byte
  232. var err error
  233. cerr := &ClusterError{}
  234. for i := pinned; i < leps+pinned; i++ {
  235. k := i % leps
  236. hc := c.clientFactory(eps[k])
  237. resp, body, err = hc.Do(ctx, action)
  238. if err != nil {
  239. cerr.Errors = append(cerr.Errors, err)
  240. // mask previous errors with context error, which is controlled by user
  241. if err == context.Canceled || err == context.DeadlineExceeded {
  242. return nil, nil, err
  243. }
  244. continue
  245. }
  246. if resp.StatusCode/100 == 5 {
  247. switch resp.StatusCode {
  248. case http.StatusInternalServerError, http.StatusServiceUnavailable:
  249. // TODO: make sure this is a no leader response
  250. cerr.Errors = append(cerr.Errors, fmt.Errorf("client: etcd member %s has no leader", eps[k].String()))
  251. default:
  252. cerr.Errors = append(cerr.Errors, fmt.Errorf("client: etcd member %s returns server error [%s]", eps[k].String(), http.StatusText(resp.StatusCode)))
  253. }
  254. continue
  255. }
  256. if k != pinned {
  257. c.Lock()
  258. c.pinned = k
  259. c.Unlock()
  260. }
  261. return resp, body, nil
  262. }
  263. return nil, nil, cerr
  264. }
  265. func (c *httpClusterClient) Endpoints() []string {
  266. c.RLock()
  267. defer c.RUnlock()
  268. eps := make([]string, len(c.endpoints))
  269. for i, ep := range c.endpoints {
  270. eps[i] = ep.String()
  271. }
  272. return eps
  273. }
  274. func (c *httpClusterClient) Sync(ctx context.Context) error {
  275. mAPI := NewMembersAPI(c)
  276. ms, err := mAPI.List(ctx)
  277. if err != nil {
  278. return err
  279. }
  280. c.Lock()
  281. defer c.Unlock()
  282. eps := make([]string, 0)
  283. for _, m := range ms {
  284. eps = append(eps, m.ClientURLs...)
  285. }
  286. sort.Sort(sort.StringSlice(eps))
  287. ceps := make([]string, len(c.endpoints))
  288. for i, cep := range c.endpoints {
  289. ceps[i] = cep.String()
  290. }
  291. sort.Sort(sort.StringSlice(ceps))
  292. // fast path if no change happens
  293. // this helps client to pin the endpoint when no cluster change
  294. if reflect.DeepEqual(eps, ceps) {
  295. return nil
  296. }
  297. return c.reset(eps)
  298. }
  299. func (c *httpClusterClient) AutoSync(ctx context.Context, interval time.Duration) error {
  300. ticker := time.NewTicker(interval)
  301. defer ticker.Stop()
  302. for {
  303. err := c.Sync(ctx)
  304. if err != nil {
  305. return err
  306. }
  307. select {
  308. case <-ctx.Done():
  309. return ctx.Err()
  310. case <-ticker.C:
  311. }
  312. }
  313. }
  314. type roundTripResponse struct {
  315. resp *http.Response
  316. err error
  317. }
  318. type simpleHTTPClient struct {
  319. transport CancelableTransport
  320. endpoint url.URL
  321. headerTimeout time.Duration
  322. }
  323. func (c *simpleHTTPClient) Do(ctx context.Context, act httpAction) (*http.Response, []byte, error) {
  324. req := act.HTTPRequest(c.endpoint)
  325. if err := printcURL(req); err != nil {
  326. return nil, nil, err
  327. }
  328. hctx, hcancel := context.WithCancel(ctx)
  329. if c.headerTimeout > 0 {
  330. hctx, hcancel = context.WithTimeout(ctx, c.headerTimeout)
  331. }
  332. defer hcancel()
  333. rtchan := make(chan roundTripResponse, 1)
  334. go func() {
  335. resp, err := c.transport.RoundTrip(req)
  336. rtchan <- roundTripResponse{resp: resp, err: err}
  337. close(rtchan)
  338. }()
  339. var resp *http.Response
  340. var err error
  341. select {
  342. case rtresp := <-rtchan:
  343. resp, err = rtresp.resp, rtresp.err
  344. case <-hctx.Done():
  345. // cancel and wait for request to actually exit before continuing
  346. c.transport.CancelRequest(req)
  347. rtresp := <-rtchan
  348. resp = rtresp.resp
  349. switch {
  350. case ctx.Err() != nil:
  351. err = ctx.Err()
  352. case hctx.Err() != nil:
  353. err = fmt.Errorf("client: endpoint %s exceeded header timeout", c.endpoint.String())
  354. default:
  355. panic("failed to get error from context")
  356. }
  357. }
  358. // always check for resp nil-ness to deal with possible
  359. // race conditions between channels above
  360. defer func() {
  361. if resp != nil {
  362. resp.Body.Close()
  363. }
  364. }()
  365. if err != nil {
  366. return nil, nil, err
  367. }
  368. var body []byte
  369. done := make(chan struct{})
  370. go func() {
  371. body, err = ioutil.ReadAll(resp.Body)
  372. done <- struct{}{}
  373. }()
  374. select {
  375. case <-ctx.Done():
  376. resp.Body.Close()
  377. <-done
  378. return nil, nil, ctx.Err()
  379. case <-done:
  380. }
  381. return resp, body, err
  382. }
  383. type authedAction struct {
  384. act httpAction
  385. credentials credentials
  386. }
  387. func (a *authedAction) HTTPRequest(url url.URL) *http.Request {
  388. r := a.act.HTTPRequest(url)
  389. r.SetBasicAuth(a.credentials.username, a.credentials.password)
  390. return r
  391. }
  392. type redirectFollowingHTTPClient struct {
  393. client httpClient
  394. checkRedirect CheckRedirectFunc
  395. }
  396. func (r *redirectFollowingHTTPClient) Do(ctx context.Context, act httpAction) (*http.Response, []byte, error) {
  397. next := act
  398. for i := 0; i < 100; i++ {
  399. if i > 0 {
  400. if err := r.checkRedirect(i); err != nil {
  401. return nil, nil, err
  402. }
  403. }
  404. resp, body, err := r.client.Do(ctx, next)
  405. if err != nil {
  406. return nil, nil, err
  407. }
  408. if resp.StatusCode/100 == 3 {
  409. hdr := resp.Header.Get("Location")
  410. if hdr == "" {
  411. return nil, nil, fmt.Errorf("Location header not set")
  412. }
  413. loc, err := url.Parse(hdr)
  414. if err != nil {
  415. return nil, nil, fmt.Errorf("Location header not valid URL: %s", hdr)
  416. }
  417. next = &redirectedHTTPAction{
  418. action: act,
  419. location: *loc,
  420. }
  421. continue
  422. }
  423. return resp, body, nil
  424. }
  425. return nil, nil, errTooManyRedirectChecks
  426. }
  427. type redirectedHTTPAction struct {
  428. action httpAction
  429. location url.URL
  430. }
  431. func (r *redirectedHTTPAction) HTTPRequest(ep url.URL) *http.Request {
  432. orig := r.action.HTTPRequest(ep)
  433. orig.URL = &r.location
  434. return orig
  435. }
  436. func shuffleEndpoints(r *rand.Rand, eps []url.URL) []url.URL {
  437. p := r.Perm(len(eps))
  438. neps := make([]url.URL, len(eps))
  439. for i, k := range p {
  440. neps[i] = eps[k]
  441. }
  442. return neps
  443. }