raftexample_test.go 2.7 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107
  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 main
  15. import (
  16. "fmt"
  17. "os"
  18. "testing"
  19. )
  20. type cluster struct {
  21. peers []string
  22. commitC []<-chan *string
  23. errorC []<-chan error
  24. proposeC []chan string
  25. }
  26. // newCluster creates a cluster of n nodes
  27. func newCluster(n int) *cluster {
  28. peers := make([]string, n)
  29. for i := range peers {
  30. peers[i] = fmt.Sprintf("http://127.0.0.1:%d", 10000+i)
  31. }
  32. clus := &cluster{
  33. peers: peers,
  34. commitC: make([]<-chan *string, len(peers)),
  35. errorC: make([]<-chan error, len(peers)),
  36. proposeC: make([]chan string, len(peers))}
  37. for i := range clus.peers {
  38. os.RemoveAll(fmt.Sprintf("raftexample-%d", i+1))
  39. clus.proposeC[i] = make(chan string, 1)
  40. clus.commitC[i], clus.errorC[i] = newRaftNode(i+1, clus.peers, clus.proposeC[i])
  41. // replay local log
  42. for s := range clus.commitC[i] {
  43. if s == nil {
  44. break
  45. }
  46. }
  47. }
  48. return clus
  49. }
  50. // Close closes all cluster nodes and returns an error if any failed.
  51. func (clus *cluster) Close() (err error) {
  52. for i := range clus.peers {
  53. close(clus.proposeC[i])
  54. for range clus.commitC[i] {
  55. // drain pending commits
  56. }
  57. // wait for channel to close
  58. if erri, _ := <-clus.errorC[i]; erri != nil {
  59. err = erri
  60. }
  61. // clean intermediates
  62. os.RemoveAll(fmt.Sprintf("raftexample-%d", i+1))
  63. }
  64. return err
  65. }
  66. // TestProposeOnCommit starts three nodes and feeds commits back into the proposal
  67. // channel. The intent is to ensure blocking on a proposal won't block raft progress.
  68. func TestProposeOnCommit(t *testing.T) {
  69. clus := newCluster(3)
  70. defer func() {
  71. if err := clus.Close(); err != nil {
  72. t.Fatal(err)
  73. }
  74. }()
  75. donec := make(chan struct{})
  76. for i := range clus.peers {
  77. // feedback for "n" committed entries, then update donec
  78. go func(pC chan<- string, cC <-chan *string, eC <-chan error) {
  79. for n := 0; n < 100; n++ {
  80. select {
  81. case s := <-cC:
  82. pC <- *s
  83. case err, _ := <-eC:
  84. t.Fatalf("eC closed (%v)", err)
  85. }
  86. }
  87. donec <- struct{}{}
  88. }(clus.proposeC[i], clus.commitC[i], clus.errorC[i])
  89. // one message feedback per node
  90. go func() { clus.proposeC[i] <- "foo" }()
  91. }
  92. for range clus.peers {
  93. <-donec
  94. }
  95. }