server_test.go 18 KB

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