server_test.go 20 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778
  1. package etcdserver
  2. import (
  3. "fmt"
  4. "math/rand"
  5. "reflect"
  6. "runtime"
  7. "sync"
  8. "testing"
  9. "time"
  10. pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
  11. "github.com/coreos/etcd/raft"
  12. "github.com/coreos/etcd/raft/raftpb"
  13. "github.com/coreos/etcd/store"
  14. "github.com/coreos/etcd/testutil"
  15. "github.com/coreos/etcd/third_party/code.google.com/p/go.net/context"
  16. )
  17. // TestDoLocalAction tests requests which do not need to go through raft to be applied,
  18. // and are served through local data.
  19. func TestDoLocalAction(t *testing.T) {
  20. tests := []struct {
  21. req pb.Request
  22. wresp Response
  23. werr error
  24. waction []string
  25. }{
  26. {
  27. pb.Request{Method: "GET", Id: 1, Wait: true},
  28. Response{Watcher: &stubWatcher{}}, nil, []string{"Watch"},
  29. },
  30. {
  31. pb.Request{Method: "GET", Id: 1},
  32. Response{Event: &store.Event{}}, nil, []string{"Get"},
  33. },
  34. {
  35. pb.Request{Method: "BADMETHOD", Id: 1},
  36. Response{}, ErrUnknownMethod, []string{},
  37. },
  38. }
  39. for i, tt := range tests {
  40. st := &storeRecorder{}
  41. srv := &EtcdServer{Store: st}
  42. resp, err := srv.Do(context.TODO(), tt.req)
  43. if err != tt.werr {
  44. t.Fatalf("#%d: err = %+v, want %+v", i, err, tt.werr)
  45. }
  46. if !reflect.DeepEqual(resp, tt.wresp) {
  47. t.Errorf("#%d: resp = %+v, want %+v", i, resp, tt.wresp)
  48. }
  49. action := st.Action()
  50. if !reflect.DeepEqual(action, tt.waction) {
  51. t.Errorf("#%d: action = %+v, want %+v", i, action, tt.waction)
  52. }
  53. }
  54. }
  55. // TestDoBadLocalAction tests server requests which do not need to go through consensus,
  56. // and return errors when they fetch from local data.
  57. func TestDoBadLocalAction(t *testing.T) {
  58. storeErr := fmt.Errorf("bah")
  59. tests := []struct {
  60. req pb.Request
  61. waction []string
  62. }{
  63. {
  64. pb.Request{Method: "GET", Id: 1, Wait: true},
  65. []string{"Watch"},
  66. },
  67. {
  68. pb.Request{Method: "GET", Id: 1},
  69. []string{"Get"},
  70. },
  71. }
  72. for i, tt := range tests {
  73. st := &errStoreRecorder{err: storeErr}
  74. srv := &EtcdServer{Store: st}
  75. resp, err := srv.Do(context.Background(), tt.req)
  76. if err != storeErr {
  77. t.Fatalf("#%d: err = %+v, want %+v", i, err, storeErr)
  78. }
  79. if !reflect.DeepEqual(resp, Response{}) {
  80. t.Errorf("#%d: resp = %+v, want %+v", i, resp, Response{})
  81. }
  82. action := st.Action()
  83. if !reflect.DeepEqual(action, tt.waction) {
  84. t.Errorf("#%d: action = %+v, want %+v", i, action, tt.waction)
  85. }
  86. }
  87. }
  88. func TestApply(t *testing.T) {
  89. tests := []struct {
  90. req pb.Request
  91. wresp Response
  92. waction []string
  93. }{
  94. {
  95. pb.Request{Method: "POST", Id: 1},
  96. Response{Event: &store.Event{}}, []string{"Create"},
  97. },
  98. {
  99. pb.Request{Method: "PUT", Id: 1, PrevExist: boolp(true), PrevIndex: 1},
  100. Response{Event: &store.Event{}}, []string{"Update"},
  101. },
  102. {
  103. pb.Request{Method: "PUT", Id: 1, PrevExist: boolp(false), PrevIndex: 1},
  104. Response{Event: &store.Event{}}, []string{"Create"},
  105. },
  106. {
  107. pb.Request{Method: "PUT", Id: 1, PrevExist: boolp(true)},
  108. Response{Event: &store.Event{}}, []string{"Update"},
  109. },
  110. {
  111. pb.Request{Method: "PUT", Id: 1, PrevExist: boolp(false)},
  112. Response{Event: &store.Event{}}, []string{"Create"},
  113. },
  114. {
  115. pb.Request{Method: "PUT", Id: 1, PrevIndex: 1},
  116. Response{Event: &store.Event{}}, []string{"CompareAndSwap"},
  117. },
  118. {
  119. pb.Request{Method: "PUT", Id: 1, PrevValue: "bar"},
  120. Response{Event: &store.Event{}}, []string{"CompareAndSwap"},
  121. },
  122. {
  123. pb.Request{Method: "PUT", Id: 1},
  124. Response{Event: &store.Event{}}, []string{"Set"},
  125. },
  126. {
  127. pb.Request{Method: "DELETE", Id: 1, PrevIndex: 1},
  128. Response{Event: &store.Event{}}, []string{"CompareAndDelete"},
  129. },
  130. {
  131. pb.Request{Method: "DELETE", Id: 1, PrevValue: "bar"},
  132. Response{Event: &store.Event{}}, []string{"CompareAndDelete"},
  133. },
  134. {
  135. pb.Request{Method: "DELETE", Id: 1},
  136. Response{Event: &store.Event{}}, []string{"Delete"},
  137. },
  138. {
  139. pb.Request{Method: "QGET", Id: 1},
  140. Response{Event: &store.Event{}}, []string{"Get"},
  141. },
  142. {
  143. pb.Request{Method: "SYNC", Id: 1},
  144. Response{}, []string{"DeleteExpiredKeys"},
  145. },
  146. {
  147. pb.Request{Method: "BADMETHOD", Id: 1},
  148. Response{err: ErrUnknownMethod}, []string{},
  149. },
  150. }
  151. for i, tt := range tests {
  152. st := &storeRecorder{}
  153. srv := &EtcdServer{Store: st}
  154. resp := srv.apply(tt.req)
  155. if !reflect.DeepEqual(resp, tt.wresp) {
  156. t.Errorf("#%d: resp = %+v, want %+v", i, resp, tt.wresp)
  157. }
  158. action := st.Action()
  159. if !reflect.DeepEqual(action, tt.waction) {
  160. t.Errorf("#%d: action = %+v, want %+v", i, action, tt.waction)
  161. }
  162. }
  163. }
  164. func TestClusterOf1(t *testing.T) { testServer(t, 1) }
  165. func TestClusterOf3(t *testing.T) { testServer(t, 3) }
  166. func testServer(t *testing.T, ns int64) {
  167. ctx, cancel := context.WithCancel(context.Background())
  168. defer cancel()
  169. ss := make([]*EtcdServer, ns)
  170. send := func(msgs []raftpb.Message) {
  171. for _, m := range msgs {
  172. t.Logf("m = %+v\n", m)
  173. ss[m.To-1].Node.Step(ctx, m)
  174. }
  175. }
  176. peers := make([]int64, ns)
  177. for i := int64(0); i < ns; i++ {
  178. peers[i] = i + 1
  179. }
  180. for i := int64(0); i < ns; i++ {
  181. id := i + 1
  182. n := raft.StartNode(id, peers, 10, 1)
  183. tk := time.NewTicker(10 * time.Millisecond)
  184. defer tk.Stop()
  185. srv := &EtcdServer{
  186. Node: n,
  187. Store: store.New(),
  188. Send: send,
  189. Storage: &storageRecorder{},
  190. Ticker: tk.C,
  191. }
  192. srv.Start()
  193. // TODO(xiangli): randomize election timeout
  194. // then remove this sleep.
  195. time.Sleep(1 * time.Millisecond)
  196. ss[i] = srv
  197. }
  198. for i := 1; i <= 10; i++ {
  199. r := pb.Request{
  200. Method: "PUT",
  201. Id: int64(i),
  202. Path: "/foo",
  203. Val: "bar",
  204. }
  205. j := rand.Intn(len(ss))
  206. t.Logf("ss = %d", j)
  207. resp, err := ss[j].Do(ctx, r)
  208. if err != nil {
  209. t.Fatal(err)
  210. }
  211. g, w := resp.Event.Node, &store.NodeExtern{
  212. Key: "/foo",
  213. ModifiedIndex: uint64(i),
  214. CreatedIndex: uint64(i),
  215. Value: stringp("bar"),
  216. }
  217. if !reflect.DeepEqual(g, w) {
  218. t.Error("value:", *g.Value)
  219. t.Errorf("g = %+v, w %+v", g, w)
  220. }
  221. }
  222. time.Sleep(10 * time.Millisecond)
  223. var last interface{}
  224. for i, sv := range ss {
  225. sv.Stop()
  226. g, _ := sv.Store.Get("/", true, true)
  227. if last != nil && !reflect.DeepEqual(last, g) {
  228. t.Errorf("server %d: Root = %#v, want %#v", i, g, last)
  229. }
  230. last = g
  231. }
  232. }
  233. func TestDoProposal(t *testing.T) {
  234. tests := []pb.Request{
  235. pb.Request{Method: "POST", Id: 1},
  236. pb.Request{Method: "PUT", Id: 1},
  237. pb.Request{Method: "DELETE", Id: 1},
  238. pb.Request{Method: "GET", Id: 1, Quorum: true},
  239. }
  240. for i, tt := range tests {
  241. ctx, _ := context.WithCancel(context.Background())
  242. n := raft.StartNode(0xBAD0, []int64{0xBAD0}, 10, 1)
  243. st := &storeRecorder{}
  244. tk := make(chan time.Time)
  245. // this makes <-tk always successful, which accelerates internal clock
  246. close(tk)
  247. srv := &EtcdServer{
  248. Node: n,
  249. Store: st,
  250. Send: func(_ []raftpb.Message) {},
  251. Storage: &storageRecorder{},
  252. Ticker: tk,
  253. }
  254. srv.Start()
  255. resp, err := srv.Do(ctx, tt)
  256. srv.Stop()
  257. action := st.Action()
  258. if len(action) != 1 {
  259. t.Errorf("#%d: len(action) = %d, want 1", i, len(action))
  260. }
  261. if err != nil {
  262. t.Fatalf("#%d: err = %v, want nil", i, err)
  263. }
  264. wresp := Response{Event: &store.Event{}}
  265. if !reflect.DeepEqual(resp, wresp) {
  266. t.Errorf("#%d: resp = %v, want %v", i, resp, wresp)
  267. }
  268. }
  269. }
  270. func TestDoProposalCancelled(t *testing.T) {
  271. ctx, cancel := context.WithCancel(context.Background())
  272. // node cannot make any progress because there are two nodes
  273. n := raft.StartNode(0xBAD0, []int64{0xBAD0, 0xBAD1}, 10, 1)
  274. st := &storeRecorder{}
  275. wait := &waitRecorder{}
  276. srv := &EtcdServer{
  277. // TODO: use fake node for better testability
  278. Node: n,
  279. Store: st,
  280. w: wait,
  281. }
  282. done := make(chan struct{})
  283. var err error
  284. go func() {
  285. _, err = srv.Do(ctx, pb.Request{Method: "PUT", Id: 1})
  286. close(done)
  287. }()
  288. cancel()
  289. <-done
  290. action := st.Action()
  291. if len(action) != 0 {
  292. t.Errorf("len(action) = %v, want 0", len(action))
  293. }
  294. if err != context.Canceled {
  295. t.Fatalf("err = %v, want %v", err, context.Canceled)
  296. }
  297. w := []string{"Register1", "Trigger1"}
  298. if !reflect.DeepEqual(wait.action, w) {
  299. t.Errorf("wait.action = %+v, want %+v", wait.action, w)
  300. }
  301. }
  302. func TestDoProposalStopped(t *testing.T) {
  303. ctx, cancel := context.WithCancel(context.Background())
  304. defer cancel()
  305. // node cannot make any progress because there are two nodes
  306. n := raft.StartNode(0xBAD0, []int64{0xBAD0, 0xBAD1}, 10, 1)
  307. st := &storeRecorder{}
  308. tk := make(chan time.Time)
  309. // this makes <-tk always successful, which accelarates internal clock
  310. close(tk)
  311. srv := &EtcdServer{
  312. // TODO: use fake node for better testability
  313. Node: n,
  314. Store: st,
  315. Send: func(_ []raftpb.Message) {},
  316. Storage: &storageRecorder{},
  317. Ticker: tk,
  318. }
  319. srv.Start()
  320. done := make(chan struct{})
  321. var err error
  322. go func() {
  323. _, err = srv.Do(ctx, pb.Request{Method: "PUT", Id: 1})
  324. close(done)
  325. }()
  326. srv.Stop()
  327. <-done
  328. action := st.Action()
  329. if len(action) != 0 {
  330. t.Errorf("len(action) = %v, want 0", len(action))
  331. }
  332. if err != ErrStopped {
  333. t.Errorf("err = %v, want %v", err, ErrStopped)
  334. }
  335. }
  336. // TestSync tests sync 1. is nonblocking 2. sends out SYNC request.
  337. func TestSync(t *testing.T) {
  338. n := raft.StartNode(0xBAD0, []int64{0xBAD0}, 10, 1)
  339. n.Campaign(context.TODO())
  340. select {
  341. case <-n.Ready():
  342. case <-time.After(time.Millisecond):
  343. t.Fatalf("expect to receive ready within 1ms, but fail")
  344. }
  345. srv := &EtcdServer{
  346. // TODO: use fake node for better testability
  347. Node: n,
  348. }
  349. start := time.Now()
  350. srv.sync(defaultSyncTimeout)
  351. // check that sync is non-blocking
  352. if d := time.Since(start); d > time.Millisecond {
  353. t.Errorf("CallSyncTime = %v, want < %v", d, time.Millisecond)
  354. }
  355. // give time for goroutine in sync to run
  356. // TODO: use fake clock
  357. var ready raft.Ready
  358. select {
  359. case ready = <-n.Ready():
  360. case <-time.After(time.Millisecond):
  361. t.Fatalf("expect to receive ready within 1ms, but fail")
  362. }
  363. if len(ready.CommittedEntries) != 1 {
  364. t.Fatalf("len(CommittedEntries) = %d, want 1", len(ready.CommittedEntries))
  365. }
  366. e := ready.CommittedEntries[0]
  367. var req pb.Request
  368. if err := req.Unmarshal(e.Data); err != nil {
  369. t.Fatalf("unmarshal error: %v", err)
  370. }
  371. if req.Method != "SYNC" {
  372. t.Errorf("method = %s, want SYNC", req.Method)
  373. }
  374. }
  375. // TestSyncFail tests the case that sync 1. is non-blocking 2. fails to
  376. // propose SYNC request because there is no leader
  377. func TestSyncFail(t *testing.T) {
  378. // The node is run without Tick and Campaign, so it has no leader forever.
  379. n := raft.StartNode(0xBAD0, []int64{0xBAD0}, 10, 1)
  380. select {
  381. case <-n.Ready():
  382. case <-time.After(time.Millisecond):
  383. t.Fatalf("expect to receive ready within 1ms, but fail")
  384. }
  385. srv := &EtcdServer{
  386. // TODO: use fake node for better testability
  387. Node: n,
  388. }
  389. routineN := runtime.NumGoroutine()
  390. start := time.Now()
  391. srv.sync(time.Millisecond)
  392. // check that sync is non-blocking
  393. if d := time.Since(start); d > time.Millisecond {
  394. t.Errorf("CallSyncTime = %v, want < %v", d, time.Millisecond)
  395. }
  396. // give time for goroutine in sync to cancel
  397. // TODO: use fake clock
  398. time.Sleep(2 * time.Millisecond)
  399. if g := runtime.NumGoroutine(); g != routineN {
  400. t.Errorf("NumGoroutine = %d, want %d", g, routineN)
  401. }
  402. select {
  403. case g := <-n.Ready():
  404. t.Errorf("ready = %+v, want no", g)
  405. default:
  406. }
  407. }
  408. func TestSyncTriggerDeleteExpriedKeys(t *testing.T) {
  409. n := raft.StartNode(0xBAD0, []int64{0xBAD0}, 10, 1)
  410. n.Campaign(context.TODO())
  411. st := &storeRecorder{}
  412. syncInterval := 5 * time.Millisecond
  413. syncTicker := time.NewTicker(syncInterval)
  414. defer syncTicker.Stop()
  415. srv := &EtcdServer{
  416. // TODO: use fake node for better testability
  417. Node: n,
  418. Store: st,
  419. Send: func(_ []raftpb.Message) {},
  420. Storage: &storageRecorder{},
  421. SyncTicker: syncTicker.C,
  422. }
  423. srv.Start()
  424. // give time for sync request to be proposed and performed
  425. // TODO: use fake clock
  426. time.Sleep(syncInterval + time.Millisecond)
  427. srv.Stop()
  428. action := st.Action()
  429. if len(action) != 1 {
  430. t.Fatalf("len(action) = %d, want 1", len(action))
  431. }
  432. if action[0] != "DeleteExpiredKeys" {
  433. t.Errorf("action = %s, want DeleteExpiredKeys", action[0])
  434. }
  435. }
  436. // snapshot should snapshot the store and cut the persistent
  437. // TODO: node.Compact is called... we need to make the node an interface
  438. func TestSnapshot(t *testing.T) {
  439. n := raft.StartNode(0xBAD0, []int64{0xBAD0}, 10, 1)
  440. defer n.Stop()
  441. st := &storeRecorder{}
  442. p := &storageRecorder{}
  443. s := &EtcdServer{
  444. Store: st,
  445. Storage: p,
  446. Node: n,
  447. }
  448. s.snapshot()
  449. action := st.Action()
  450. if len(action) != 1 {
  451. t.Fatalf("len(action) = %d, want 1", len(action))
  452. }
  453. if action[0] != "Save" {
  454. t.Errorf("action = %s, want Save", action[0])
  455. }
  456. action = p.Action()
  457. if len(action) != 1 {
  458. t.Fatalf("len(action) = %d, want 1", len(action))
  459. }
  460. if action[0] != "Cut" {
  461. t.Errorf("action = %s, want Cut", action[0])
  462. }
  463. }
  464. // Applied > SnapCount should trigger a SaveSnap event
  465. func TestTriggerSnap(t *testing.T) {
  466. ctx := context.Background()
  467. n := raft.StartNode(0xBAD0, []int64{0xBAD0}, 10, 1)
  468. n.Campaign(ctx)
  469. st := &storeRecorder{}
  470. p := &storageRecorder{}
  471. s := &EtcdServer{
  472. Store: st,
  473. Send: func(_ []raftpb.Message) {},
  474. Storage: p,
  475. Node: n,
  476. SnapCount: 10,
  477. }
  478. s.Start()
  479. for i := 0; int64(i) < s.SnapCount; i++ {
  480. s.Do(ctx, pb.Request{Method: "PUT", Id: 1})
  481. }
  482. time.Sleep(time.Millisecond)
  483. s.Stop()
  484. action := p.Action()
  485. // each operation is recorded as a Save
  486. // Nop + SnapCount * Puts + Cut + SaveSnap = Save + SnapCount * Save + Cut + SaveSnap
  487. if len(action) != 3+int(s.SnapCount) {
  488. t.Fatalf("len(action) = %d, want %d", len(action), 3+int(s.SnapCount))
  489. }
  490. if action[12] != "SaveSnap" {
  491. t.Errorf("action = %s, want SaveSnap", action[12])
  492. }
  493. }
  494. // TestRecvSnapshot tests when it receives a snapshot from raft leader,
  495. // it should trigger storage.SaveSnap and also store.Recover.
  496. func TestRecvSnapshot(t *testing.T) {
  497. n := newReadyNode()
  498. st := &storeRecorder{}
  499. p := &storageRecorder{}
  500. s := &EtcdServer{
  501. Store: st,
  502. Send: func(_ []raftpb.Message) {},
  503. Storage: p,
  504. Node: n,
  505. }
  506. s.Start()
  507. n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Index: 1}}
  508. // make goroutines move forward to receive snapshot
  509. testutil.ForceGosched()
  510. s.Stop()
  511. waction := []string{"Recovery"}
  512. if g := st.Action(); !reflect.DeepEqual(g, waction) {
  513. t.Errorf("store action = %v, want %v", g, waction)
  514. }
  515. waction = []string{"Save", "SaveSnap"}
  516. if g := p.Action(); !reflect.DeepEqual(g, waction) {
  517. t.Errorf("storage action = %v, want %v", g, waction)
  518. }
  519. }
  520. // TestRecvSlowSnapshot tests that slow snapshot will not be applied
  521. // to store.
  522. func TestRecvSlowSnapshot(t *testing.T) {
  523. n := newReadyNode()
  524. st := &storeRecorder{}
  525. s := &EtcdServer{
  526. Store: st,
  527. Send: func(_ []raftpb.Message) {},
  528. Storage: &storageRecorder{},
  529. Node: n,
  530. }
  531. s.Start()
  532. n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Index: 1}}
  533. // make goroutines move forward to receive snapshot
  534. testutil.ForceGosched()
  535. action := st.Action()
  536. n.readyc <- raft.Ready{Snapshot: raftpb.Snapshot{Index: 1}}
  537. // make goroutines move forward to receive snapshot
  538. testutil.ForceGosched()
  539. s.Stop()
  540. if g := st.Action(); !reflect.DeepEqual(g, action) {
  541. t.Errorf("store action = %v, want %v", g, action)
  542. }
  543. }
  544. // TODO: test wait trigger correctness in multi-server case
  545. func TestGetBool(t *testing.T) {
  546. tests := []struct {
  547. b *bool
  548. wb bool
  549. wset bool
  550. }{
  551. {nil, false, false},
  552. {boolp(true), true, true},
  553. {boolp(false), false, true},
  554. }
  555. for i, tt := range tests {
  556. b, set := getBool(tt.b)
  557. if b != tt.wb {
  558. t.Errorf("#%d: value = %v, want %v", i, b, tt.wb)
  559. }
  560. if set != tt.wset {
  561. t.Errorf("#%d: set = %v, want %v", i, set, tt.wset)
  562. }
  563. }
  564. }
  565. type recorder struct {
  566. sync.Mutex
  567. action []string
  568. }
  569. func (r *recorder) record(action string) {
  570. r.Lock()
  571. r.action = append(r.action, action)
  572. r.Unlock()
  573. }
  574. func (r *recorder) Action() []string {
  575. r.Lock()
  576. cpy := make([]string, len(r.action))
  577. copy(cpy, r.action)
  578. r.Unlock()
  579. return cpy
  580. }
  581. type storeRecorder struct {
  582. recorder
  583. }
  584. func (s *storeRecorder) Version() int { return 0 }
  585. func (s *storeRecorder) Index() uint64 { return 0 }
  586. func (s *storeRecorder) Get(_ string, _, _ bool) (*store.Event, error) {
  587. s.record("Get")
  588. return &store.Event{}, nil
  589. }
  590. func (s *storeRecorder) Set(_ string, _ bool, _ string, _ time.Time) (*store.Event, error) {
  591. s.record("Set")
  592. return &store.Event{}, nil
  593. }
  594. func (s *storeRecorder) Update(_, _ string, _ time.Time) (*store.Event, error) {
  595. s.record("Update")
  596. return &store.Event{}, nil
  597. }
  598. func (s *storeRecorder) Create(_ string, _ bool, _ string, _ bool, _ time.Time) (*store.Event, error) {
  599. s.record("Create")
  600. return &store.Event{}, nil
  601. }
  602. func (s *storeRecorder) CompareAndSwap(_, _ string, _ uint64, _ string, _ time.Time) (*store.Event, error) {
  603. s.record("CompareAndSwap")
  604. return &store.Event{}, nil
  605. }
  606. func (s *storeRecorder) Delete(_ string, _, _ bool) (*store.Event, error) {
  607. s.record("Delete")
  608. return &store.Event{}, nil
  609. }
  610. func (s *storeRecorder) CompareAndDelete(_, _ string, _ uint64) (*store.Event, error) {
  611. s.record("CompareAndDelete")
  612. return &store.Event{}, nil
  613. }
  614. func (s *storeRecorder) Watch(_ string, _, _ bool, _ uint64) (store.Watcher, error) {
  615. s.record("Watch")
  616. return &stubWatcher{}, nil
  617. }
  618. func (s *storeRecorder) Save() ([]byte, error) {
  619. s.record("Save")
  620. return nil, nil
  621. }
  622. func (s *storeRecorder) Recovery(b []byte) error {
  623. s.record("Recovery")
  624. return nil
  625. }
  626. func (s *storeRecorder) TotalTransactions() uint64 { return 0 }
  627. func (s *storeRecorder) JsonStats() []byte { return nil }
  628. func (s *storeRecorder) DeleteExpiredKeys(cutoff time.Time) {
  629. s.record("DeleteExpiredKeys")
  630. }
  631. type stubWatcher struct{}
  632. func (w *stubWatcher) EventChan() chan *store.Event { return nil }
  633. func (w *stubWatcher) Remove() {}
  634. // errStoreRecorder returns an store error on Get, Watch request
  635. type errStoreRecorder struct {
  636. storeRecorder
  637. err error
  638. }
  639. func (s *errStoreRecorder) Get(_ string, _, _ bool) (*store.Event, error) {
  640. s.record("Get")
  641. return nil, s.err
  642. }
  643. func (s *errStoreRecorder) Watch(_ string, _, _ bool, _ uint64) (store.Watcher, error) {
  644. s.record("Watch")
  645. return nil, s.err
  646. }
  647. type waitRecorder struct {
  648. action []string
  649. }
  650. func (w *waitRecorder) Register(id int64) <-chan interface{} {
  651. w.action = append(w.action, fmt.Sprint("Register", id))
  652. return nil
  653. }
  654. func (w *waitRecorder) Trigger(id int64, x interface{}) {
  655. w.action = append(w.action, fmt.Sprint("Trigger", id))
  656. }
  657. func boolp(b bool) *bool { return &b }
  658. func stringp(s string) *string { return &s }
  659. type storageRecorder struct {
  660. recorder
  661. }
  662. func (p *storageRecorder) Save(st raftpb.HardState, ents []raftpb.Entry) {
  663. p.record("Save")
  664. }
  665. func (p *storageRecorder) Cut() error {
  666. p.record("Cut")
  667. return nil
  668. }
  669. func (p *storageRecorder) SaveSnap(st raftpb.Snapshot) {
  670. if raft.IsEmptySnap(st) {
  671. return
  672. }
  673. p.record("SaveSnap")
  674. }
  675. type readyNode struct {
  676. readyc chan raft.Ready
  677. }
  678. func newReadyNode() *readyNode {
  679. readyc := make(chan raft.Ready, 1)
  680. return &readyNode{readyc: readyc}
  681. }
  682. func (n *readyNode) Tick() {}
  683. func (n *readyNode) Campaign(ctx context.Context) error { return nil }
  684. func (n *readyNode) Propose(ctx context.Context, data []byte) error { return nil }
  685. func (n *readyNode) Configure(ctx context.Context, data []byte) error { return nil }
  686. func (n *readyNode) Step(ctx context.Context, msg raftpb.Message) error { return nil }
  687. func (n *readyNode) Ready() <-chan raft.Ready { return n.readyc }
  688. func (n *readyNode) Stop() {}
  689. func (n *readyNode) Compact(d []byte) {}
  690. func (n *readyNode) AddNode(id int64) {}
  691. func (n *readyNode) RemoveNode(id int64) {}
  692. func TestGenID(t *testing.T) {
  693. // Sanity check that the GenID function has been seeded appropriately
  694. // (math/rand is seeded with 1 by default)
  695. r := rand.NewSource(int64(1))
  696. var n int64
  697. for n == 0 {
  698. n = r.Int63()
  699. }
  700. if n == GenID() {
  701. t.Fatalf("GenID's rand seeded with 1!")
  702. }
  703. }