pipeline_test.go 8.4 KB

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