log.go 4.4 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209
  1. package raft
  2. import "fmt"
  3. const (
  4. Normal int64 = iota
  5. ClusterInit
  6. AddNode
  7. RemoveNode
  8. )
  9. const (
  10. defaultCompactThreshold = 10000
  11. )
  12. type Entry struct {
  13. Type int64
  14. Term int64
  15. Data []byte
  16. }
  17. func (e *Entry) isConfig() bool {
  18. return e.Type == AddNode || e.Type == RemoveNode
  19. }
  20. type raftLog struct {
  21. ents []Entry
  22. unstable int64
  23. committed int64
  24. applied int64
  25. offset int64
  26. // want a compact after the number of entries exceeds the threshold
  27. // TODO(xiangli) size might be a better criteria
  28. compactThreshold int64
  29. }
  30. func newLog() *raftLog {
  31. return &raftLog{
  32. ents: make([]Entry, 1),
  33. unstable: 1,
  34. committed: 0,
  35. applied: 0,
  36. compactThreshold: defaultCompactThreshold,
  37. }
  38. }
  39. func (l *raftLog) String() string {
  40. return fmt.Sprintf("offset=%d committed=%d applied=%d len(ents)=%d", l.offset, l.committed, l.applied, len(l.ents))
  41. }
  42. func (l *raftLog) maybeAppend(index, logTerm, committed int64, ents ...Entry) bool {
  43. if l.matchTerm(index, logTerm) {
  44. from := index + 1
  45. ci := l.findConflict(from, ents)
  46. switch {
  47. case ci == -1:
  48. case ci <= l.committed:
  49. panic("conflict with committed entry")
  50. default:
  51. l.append(ci-1, ents[ci-from:]...)
  52. }
  53. if l.committed < committed {
  54. l.committed = min(committed, l.lastIndex())
  55. }
  56. return true
  57. }
  58. return false
  59. }
  60. func (l *raftLog) append(after int64, ents ...Entry) int64 {
  61. l.ents = append(l.slice(l.offset, after+1), ents...)
  62. l.unstable = min(l.unstable, after+1)
  63. return l.lastIndex()
  64. }
  65. func (l *raftLog) findConflict(from int64, ents []Entry) int64 {
  66. for i, ne := range ents {
  67. if oe := l.at(from + int64(i)); oe == nil || oe.Term != ne.Term {
  68. return from + int64(i)
  69. }
  70. }
  71. return -1
  72. }
  73. func (l *raftLog) unstableEnts() (int64, []Entry) {
  74. offset := l.unstable
  75. ents := l.entries(l.unstable)
  76. l.unstable = l.lastIndex() + 1
  77. return offset, ents
  78. }
  79. func (l *raftLog) lastIndex() int64 {
  80. return int64(len(l.ents)) - 1 + l.offset
  81. }
  82. func (l *raftLog) term(i int64) int64 {
  83. if e := l.at(i); e != nil {
  84. return e.Term
  85. }
  86. return -1
  87. }
  88. func (l *raftLog) entries(i int64) []Entry {
  89. // never send out the first entry
  90. // first entry is only used for matching
  91. // prevLogTerm
  92. if i == l.offset {
  93. panic("cannot return the first entry in log")
  94. }
  95. return l.slice(i, l.lastIndex()+1)
  96. }
  97. func (l *raftLog) isUpToDate(i, term int64) bool {
  98. e := l.at(l.lastIndex())
  99. return term > e.Term || (term == e.Term && i >= l.lastIndex())
  100. }
  101. func (l *raftLog) matchTerm(i, term int64) bool {
  102. if e := l.at(i); e != nil {
  103. return e.Term == term
  104. }
  105. return false
  106. }
  107. func (l *raftLog) maybeCommit(maxIndex, term int64) bool {
  108. if maxIndex > l.committed && l.term(maxIndex) == term {
  109. l.committed = maxIndex
  110. return true
  111. }
  112. return false
  113. }
  114. // nextEnts returns all the available entries for execution.
  115. // all the returned entries will be marked as applied.
  116. func (l *raftLog) nextEnts() (ents []Entry) {
  117. if l.committed > l.applied {
  118. ents = l.slice(l.applied+1, l.committed+1)
  119. l.applied = l.committed
  120. }
  121. return ents
  122. }
  123. // compact compacts all log entries until i.
  124. // It removes the log entries before i, exclusive.
  125. // i must be not smaller than the index of the first entry
  126. // and not greater than the index of the last entry.
  127. // the number of entries after compaction will be returned.
  128. func (l *raftLog) compact(i int64) int64 {
  129. if l.isOutOfBounds(i) {
  130. panic(fmt.Sprintf("compact %d out of bounds [%d:%d]", i, l.offset, l.lastIndex()))
  131. }
  132. l.ents = l.slice(i, l.lastIndex()+1)
  133. l.unstable = max(i+1, l.unstable)
  134. l.offset = i
  135. return int64(len(l.ents))
  136. }
  137. func (l *raftLog) shouldCompact() bool {
  138. return (l.applied - l.offset) > l.compactThreshold
  139. }
  140. func (l *raftLog) restore(index, term int64) {
  141. l.ents = []Entry{{Term: term}}
  142. l.unstable = index + 1
  143. l.committed = index
  144. l.applied = index
  145. l.offset = index
  146. }
  147. func (l *raftLog) at(i int64) *Entry {
  148. if l.isOutOfBounds(i) {
  149. return nil
  150. }
  151. return &l.ents[i-l.offset]
  152. }
  153. // slice get a slice of log entries from lo through hi-1, inclusive.
  154. func (l *raftLog) slice(lo int64, hi int64) []Entry {
  155. if lo >= hi {
  156. return nil
  157. }
  158. if l.isOutOfBounds(lo) || l.isOutOfBounds(hi-1) {
  159. return nil
  160. }
  161. return l.ents[lo-l.offset : hi-l.offset]
  162. }
  163. func (l *raftLog) isOutOfBounds(i int64) bool {
  164. if i < l.offset || i > l.lastIndex() {
  165. return true
  166. }
  167. return false
  168. }
  169. func min(a, b int64) int64 {
  170. if a > b {
  171. return b
  172. }
  173. return a
  174. }
  175. func max(a, b int64) int64 {
  176. if a > b {
  177. return a
  178. }
  179. return b
  180. }