pipeline_test.go 8.8 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310
  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 rafthttp
  15. import (
  16. "errors"
  17. "fmt"
  18. "io"
  19. "io/ioutil"
  20. "net/http"
  21. "sync"
  22. "testing"
  23. "time"
  24. "github.com/coreos/etcd/etcdserver/stats"
  25. "github.com/coreos/etcd/pkg/testutil"
  26. "github.com/coreos/etcd/pkg/types"
  27. "github.com/coreos/etcd/raft/raftpb"
  28. "github.com/coreos/etcd/version"
  29. )
  30. // TestPipelineSend tests that pipeline could send data using roundtripper
  31. // and increase success count in stats.
  32. func TestPipelineSend(t *testing.T) {
  33. tr := &roundTripperRecorder{}
  34. picker := mustNewURLPicker(t, []string{"http://localhost:2380"})
  35. fs := &stats.FollowerStats{}
  36. tp := &Transport{pipelineRt: tr}
  37. p := newPipeline(tp, picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), fs, &fakeRaft{}, nil)
  38. p.msgc <- raftpb.Message{Type: raftpb.MsgApp}
  39. testutil.WaitSchedule()
  40. p.stop()
  41. if tr.Request() == nil {
  42. t.Errorf("sender fails to post the data")
  43. }
  44. fs.Lock()
  45. defer fs.Unlock()
  46. if fs.Counts.Success != 1 {
  47. t.Errorf("success = %d, want 1", fs.Counts.Success)
  48. }
  49. }
  50. // TestPipelineKeepSendingWhenPostError tests that pipeline can keep
  51. // sending messages if previous messages meet post error.
  52. func TestPipelineKeepSendingWhenPostError(t *testing.T) {
  53. tr := &respRoundTripper{err: fmt.Errorf("roundtrip error")}
  54. picker := mustNewURLPicker(t, []string{"http://localhost:2380"})
  55. fs := &stats.FollowerStats{}
  56. tp := &Transport{pipelineRt: tr}
  57. p := newPipeline(tp, picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), fs, &fakeRaft{}, nil)
  58. for i := 0; i < 50; i++ {
  59. p.msgc <- raftpb.Message{Type: raftpb.MsgApp}
  60. }
  61. testutil.WaitSchedule()
  62. p.stop()
  63. // check it send out 50 requests
  64. tr.mu.Lock()
  65. defer tr.mu.Unlock()
  66. if tr.reqCount != 50 {
  67. t.Errorf("request count = %d, want 50", tr.reqCount)
  68. }
  69. }
  70. func TestPipelineExceedMaximumServing(t *testing.T) {
  71. tr := newRoundTripperBlocker()
  72. picker := mustNewURLPicker(t, []string{"http://localhost:2380"})
  73. fs := &stats.FollowerStats{}
  74. tp := &Transport{pipelineRt: tr}
  75. p := newPipeline(tp, picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), fs, &fakeRaft{}, nil)
  76. // keep the sender busy and make the buffer full
  77. // nothing can go out as we block the sender
  78. testutil.WaitSchedule()
  79. for i := 0; i < connPerPipeline+pipelineBufSize; i++ {
  80. select {
  81. case p.msgc <- raftpb.Message{}:
  82. default:
  83. t.Errorf("failed to send out message")
  84. }
  85. // force the sender to grab data
  86. testutil.WaitSchedule()
  87. }
  88. // try to send a data when we are sure the buffer is full
  89. select {
  90. case p.msgc <- raftpb.Message{}:
  91. t.Errorf("unexpected message sendout")
  92. default:
  93. }
  94. // unblock the senders and force them to send out the data
  95. tr.unblock()
  96. testutil.WaitSchedule()
  97. // It could send new data after previous ones succeed
  98. select {
  99. case p.msgc <- raftpb.Message{}:
  100. default:
  101. t.Errorf("failed to send out message")
  102. }
  103. p.stop()
  104. }
  105. // TestPipelineSendFailed tests that when send func meets the post error,
  106. // it increases fail count in stats.
  107. func TestPipelineSendFailed(t *testing.T) {
  108. picker := mustNewURLPicker(t, []string{"http://localhost:2380"})
  109. fs := &stats.FollowerStats{}
  110. tp := &Transport{pipelineRt: newRespRoundTripper(0, errors.New("blah"))}
  111. p := newPipeline(tp, picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), fs, &fakeRaft{}, nil)
  112. p.msgc <- raftpb.Message{Type: raftpb.MsgApp}
  113. testutil.WaitSchedule()
  114. p.stop()
  115. fs.Lock()
  116. defer fs.Unlock()
  117. if fs.Counts.Fail != 1 {
  118. t.Errorf("fail = %d, want 1", fs.Counts.Fail)
  119. }
  120. }
  121. func TestPipelinePost(t *testing.T) {
  122. tr := &roundTripperRecorder{}
  123. picker := mustNewURLPicker(t, []string{"http://localhost:2380"})
  124. tp := &Transport{pipelineRt: tr}
  125. p := newPipeline(tp, picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), nil, &fakeRaft{}, nil)
  126. if err := p.post([]byte("some data")); err != nil {
  127. t.Fatalf("unexpected post error: %v", err)
  128. }
  129. p.stop()
  130. if g := tr.Request().Method; g != "POST" {
  131. t.Errorf("method = %s, want %s", g, "POST")
  132. }
  133. if g := tr.Request().URL.String(); g != "http://localhost:2380/raft" {
  134. t.Errorf("url = %s, want %s", g, "http://localhost:2380/raft")
  135. }
  136. if g := tr.Request().Header.Get("Content-Type"); g != "application/protobuf" {
  137. t.Errorf("content type = %s, want %s", g, "application/protobuf")
  138. }
  139. if g := tr.Request().Header.Get("X-Server-Version"); g != version.Version {
  140. t.Errorf("version = %s, want %s", g, version.Version)
  141. }
  142. if g := tr.Request().Header.Get("X-Min-Cluster-Version"); g != version.MinClusterVersion {
  143. t.Errorf("min version = %s, want %s", g, version.MinClusterVersion)
  144. }
  145. if g := tr.Request().Header.Get("X-Etcd-Cluster-ID"); g != "1" {
  146. t.Errorf("cluster id = %s, want %s", g, "1")
  147. }
  148. b, err := ioutil.ReadAll(tr.Request().Body)
  149. if err != nil {
  150. t.Fatalf("unexpected ReadAll error: %v", err)
  151. }
  152. if string(b) != "some data" {
  153. t.Errorf("body = %s, want %s", b, "some data")
  154. }
  155. }
  156. func TestPipelinePostBad(t *testing.T) {
  157. tests := []struct {
  158. u string
  159. code int
  160. err error
  161. }{
  162. // RoundTrip returns error
  163. {"http://localhost:2380", 0, errors.New("blah")},
  164. // unexpected response status code
  165. {"http://localhost:2380", http.StatusOK, nil},
  166. {"http://localhost:2380", http.StatusCreated, nil},
  167. }
  168. for i, tt := range tests {
  169. picker := mustNewURLPicker(t, []string{tt.u})
  170. tp := &Transport{pipelineRt: newRespRoundTripper(tt.code, tt.err)}
  171. p := newPipeline(tp, picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), nil, &fakeRaft{}, make(chan error))
  172. err := p.post([]byte("some data"))
  173. p.stop()
  174. if err == nil {
  175. t.Errorf("#%d: err = nil, want not nil", i)
  176. }
  177. }
  178. }
  179. func TestPipelinePostErrorc(t *testing.T) {
  180. tests := []struct {
  181. u string
  182. code int
  183. err error
  184. }{
  185. {"http://localhost:2380", http.StatusForbidden, nil},
  186. }
  187. for i, tt := range tests {
  188. picker := mustNewURLPicker(t, []string{tt.u})
  189. errorc := make(chan error, 1)
  190. tp := &Transport{pipelineRt: newRespRoundTripper(tt.code, tt.err)}
  191. p := newPipeline(tp, picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), nil, &fakeRaft{}, errorc)
  192. p.post([]byte("some data"))
  193. p.stop()
  194. select {
  195. case <-errorc:
  196. default:
  197. t.Fatalf("#%d: cannot receive from errorc", i)
  198. }
  199. }
  200. }
  201. func TestStopBlockedPipeline(t *testing.T) {
  202. picker := mustNewURLPicker(t, []string{"http://localhost:2380"})
  203. tp := &Transport{pipelineRt: newRoundTripperBlocker()}
  204. p := newPipeline(tp, picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), nil, &fakeRaft{}, nil)
  205. // send many messages that most of them will be blocked in buffer
  206. for i := 0; i < connPerPipeline*10; i++ {
  207. p.msgc <- raftpb.Message{}
  208. }
  209. done := make(chan struct{})
  210. go func() {
  211. p.stop()
  212. done <- struct{}{}
  213. }()
  214. select {
  215. case <-done:
  216. case <-time.After(time.Second):
  217. t.Fatalf("failed to stop pipeline in 1s")
  218. }
  219. }
  220. type roundTripperBlocker struct {
  221. unblockc chan struct{}
  222. mu sync.Mutex
  223. cancel map[*http.Request]chan struct{}
  224. }
  225. func newRoundTripperBlocker() *roundTripperBlocker {
  226. return &roundTripperBlocker{
  227. unblockc: make(chan struct{}),
  228. cancel: make(map[*http.Request]chan struct{}),
  229. }
  230. }
  231. func (t *roundTripperBlocker) unblock() {
  232. close(t.unblockc)
  233. }
  234. func (t *roundTripperBlocker) CancelRequest(req *http.Request) {
  235. t.mu.Lock()
  236. defer t.mu.Unlock()
  237. if c, ok := t.cancel[req]; ok {
  238. c <- struct{}{}
  239. delete(t.cancel, req)
  240. }
  241. }
  242. type respRoundTripper struct {
  243. mu sync.Mutex
  244. reqCount int
  245. code int
  246. header http.Header
  247. err error
  248. }
  249. func newRespRoundTripper(code int, err error) *respRoundTripper {
  250. return &respRoundTripper{code: code, err: err}
  251. }
  252. func (t *respRoundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
  253. t.mu.Lock()
  254. defer t.mu.Unlock()
  255. t.reqCount++
  256. return &http.Response{StatusCode: t.code, Header: t.header, Body: &nopReadCloser{}}, t.err
  257. }
  258. type roundTripperRecorder struct {
  259. req *http.Request
  260. sync.Mutex
  261. }
  262. func (t *roundTripperRecorder) RoundTrip(req *http.Request) (*http.Response, error) {
  263. t.Lock()
  264. defer t.Unlock()
  265. t.req = req
  266. return &http.Response{StatusCode: http.StatusNoContent, Body: &nopReadCloser{}}, nil
  267. }
  268. func (t *roundTripperRecorder) Request() *http.Request {
  269. t.Lock()
  270. defer t.Unlock()
  271. return t.req
  272. }
  273. type nopReadCloser struct{}
  274. func (n *nopReadCloser) Read(p []byte) (int, error) { return 0, io.EOF }
  275. func (n *nopReadCloser) Close() error { return nil }