server_test.go 17 KB

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