log_test.go 17 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622
  1. /*
  2. Copyright 2014 CoreOS, Inc.
  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. http://www.apache.org/licenses/LICENSE-2.0
  7. Unless required by applicable law or agreed to in writing, software
  8. distributed under the License is distributed on an "AS IS" BASIS,
  9. WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  10. See the License for the specific language governing permissions and
  11. limitations under the License.
  12. */
  13. package raft
  14. import (
  15. "reflect"
  16. "testing"
  17. pb "github.com/coreos/etcd/raft/raftpb"
  18. )
  19. func TestFindConflict(t *testing.T) {
  20. previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}
  21. tests := []struct {
  22. from uint64
  23. ents []pb.Entry
  24. wconflict uint64
  25. }{
  26. // no conflict, empty ent
  27. {1, []pb.Entry{}, 0},
  28. {3, []pb.Entry{}, 0},
  29. // no conflict
  30. {1, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}, 0},
  31. {2, []pb.Entry{{Index: 2, Term: 2}, {Index: 3, Term: 3}}, 0},
  32. {3, []pb.Entry{{Index: 3, Term: 3}}, 0},
  33. // no conflict, but has new entries
  34. {1, []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
  35. {2, []pb.Entry{{Index: 2, Term: 2}, {Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
  36. {3, []pb.Entry{{Index: 3, Term: 3}, {Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
  37. {4, []pb.Entry{{Index: 4, Term: 4}, {Index: 5, Term: 4}}, 4},
  38. // conflicts with existing entries
  39. {1, []pb.Entry{{Index: 1, Term: 4}, {Index: 2, Term: 4}}, 1},
  40. {2, []pb.Entry{{Index: 2, Term: 1}, {Index: 3, Term: 4}, {Index: 4, Term: 4}}, 2},
  41. {3, []pb.Entry{{Index: 3, Term: 1}, {Index: 4, Term: 2}, {Index: 5, Term: 4}, {Index: 6, Term: 4}}, 3},
  42. }
  43. for i, tt := range tests {
  44. raftLog := newLog(NewMemoryStorage())
  45. raftLog.append(raftLog.lastIndex(), previousEnts...)
  46. gconflict := raftLog.findConflict(tt.from, tt.ents)
  47. if gconflict != tt.wconflict {
  48. t.Errorf("#%d: conflict = %d, want %d", i, gconflict, tt.wconflict)
  49. }
  50. }
  51. }
  52. func TestIsUpToDate(t *testing.T) {
  53. previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}
  54. raftLog := newLog(NewMemoryStorage())
  55. raftLog.append(raftLog.lastIndex(), previousEnts...)
  56. tests := []struct {
  57. lastIndex uint64
  58. term uint64
  59. wUpToDate bool
  60. }{
  61. // greater term, ignore lastIndex
  62. {raftLog.lastIndex() - 1, 4, true},
  63. {raftLog.lastIndex(), 4, true},
  64. {raftLog.lastIndex() + 1, 4, true},
  65. // smaller term, ignore lastIndex
  66. {raftLog.lastIndex() - 1, 2, false},
  67. {raftLog.lastIndex(), 2, false},
  68. {raftLog.lastIndex() + 1, 2, false},
  69. // equal term, lager lastIndex wins
  70. {raftLog.lastIndex() - 1, 3, false},
  71. {raftLog.lastIndex(), 3, true},
  72. {raftLog.lastIndex() + 1, 3, true},
  73. }
  74. for i, tt := range tests {
  75. gUpToDate := raftLog.isUpToDate(tt.lastIndex, tt.term)
  76. if gUpToDate != tt.wUpToDate {
  77. t.Errorf("#%d: uptodate = %v, want %v", i, gUpToDate, tt.wUpToDate)
  78. }
  79. }
  80. }
  81. func TestAppend(t *testing.T) {
  82. previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}}
  83. tests := []struct {
  84. after uint64
  85. ents []pb.Entry
  86. windex uint64
  87. wents []pb.Entry
  88. wunstable uint64
  89. }{
  90. {
  91. 2,
  92. []pb.Entry{},
  93. 2,
  94. []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}},
  95. 3,
  96. },
  97. {
  98. 2,
  99. []pb.Entry{{Index: 3, Term: 2}},
  100. 3,
  101. []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 2}},
  102. 3,
  103. },
  104. // conflicts with index 1
  105. {
  106. 0,
  107. []pb.Entry{{Index: 1, Term: 2}},
  108. 1,
  109. []pb.Entry{{Index: 1, Term: 2}},
  110. 1,
  111. },
  112. // conflicts with index 2
  113. {
  114. 1,
  115. []pb.Entry{{Index: 2, Term: 3}, {Index: 3, Term: 3}},
  116. 3,
  117. []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 3}, {Index: 3, Term: 3}},
  118. 2,
  119. },
  120. }
  121. for i, tt := range tests {
  122. storage := NewMemoryStorage()
  123. storage.Append(previousEnts)
  124. raftLog := newLog(storage)
  125. index := raftLog.append(tt.after, tt.ents...)
  126. if index != tt.windex {
  127. t.Errorf("#%d: lastIndex = %d, want %d", i, index, tt.windex)
  128. }
  129. if g := raftLog.entries(1); !reflect.DeepEqual(g, tt.wents) {
  130. t.Errorf("#%d: logEnts = %+v, want %+v", i, g, tt.wents)
  131. }
  132. if g := raftLog.unstable; g != tt.wunstable {
  133. t.Errorf("#%d: unstable = %d, want %d", i, g, tt.wunstable)
  134. }
  135. }
  136. }
  137. // TestLogMaybeAppend ensures:
  138. // If the given (index, term) matches with the existing log:
  139. // 1. If an existing entry conflicts with a new one (same index
  140. // but different terms), delete the existing entry and all that
  141. // follow it
  142. // 2.Append any new entries not already in the log
  143. // If the given (index, term) does not match with the existing log:
  144. // return false
  145. func TestLogMaybeAppend(t *testing.T) {
  146. previousEnts := []pb.Entry{{Index: 1, Term: 1}, {Index: 2, Term: 2}, {Index: 3, Term: 3}}
  147. lastindex := uint64(3)
  148. lastterm := uint64(3)
  149. commit := uint64(1)
  150. tests := []struct {
  151. logTerm uint64
  152. index uint64
  153. committed uint64
  154. ents []pb.Entry
  155. wlasti uint64
  156. wappend bool
  157. wcommit uint64
  158. wpanic bool
  159. }{
  160. // not match: term is different
  161. {
  162. lastterm - 1, lastindex, lastindex, []pb.Entry{{Index: lastindex + 1, Term: 4}},
  163. 0, false, commit, false,
  164. },
  165. // not match: index out of bound
  166. {
  167. lastterm, lastindex + 1, lastindex, []pb.Entry{{Index: lastindex + 2, Term: 4}},
  168. 0, false, commit, false,
  169. },
  170. // match with the last existing entry
  171. {
  172. lastterm, lastindex, lastindex, nil,
  173. lastindex, true, lastindex, false,
  174. },
  175. {
  176. lastterm, lastindex, lastindex + 1, nil,
  177. lastindex, true, lastindex, false, // do not increase commit higher than lastnewi
  178. },
  179. {
  180. lastterm, lastindex, lastindex - 1, nil,
  181. lastindex, true, lastindex - 1, false, // commit up to the commit in the message
  182. },
  183. {
  184. lastterm, lastindex, 0, nil,
  185. lastindex, true, commit, false, // commit do not decrease
  186. },
  187. {
  188. 0, 0, lastindex, nil,
  189. 0, true, commit, false, // commit do not decrease
  190. },
  191. {
  192. lastterm, lastindex, lastindex, []pb.Entry{{Index: lastindex + 1, Term: 4}},
  193. lastindex + 1, true, lastindex, false,
  194. },
  195. {
  196. lastterm, lastindex, lastindex + 1, []pb.Entry{{Index: lastindex + 1, Term: 4}},
  197. lastindex + 1, true, lastindex + 1, false,
  198. },
  199. {
  200. lastterm, lastindex, lastindex + 2, []pb.Entry{{Index: lastindex + 1, Term: 4}},
  201. lastindex + 1, true, lastindex + 1, false, // do not increase commit higher than lastnewi
  202. },
  203. {
  204. lastterm, lastindex, lastindex + 2, []pb.Entry{{Index: lastindex + 1, Term: 4}, {Index: lastindex + 2, Term: 4}},
  205. lastindex + 2, true, lastindex + 2, false,
  206. },
  207. // match with the the entry in the middle
  208. {
  209. lastterm - 1, lastindex - 1, lastindex, []pb.Entry{{Index: lastindex, Term: 4}},
  210. lastindex, true, lastindex, false,
  211. },
  212. {
  213. lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Index: lastindex - 1, Term: 4}},
  214. lastindex - 1, true, lastindex - 1, false,
  215. },
  216. {
  217. lastterm - 3, lastindex - 3, lastindex, []pb.Entry{{Index: lastindex - 2, Term: 4}},
  218. lastindex - 2, true, lastindex - 2, true, // conflict with existing committed entry
  219. },
  220. {
  221. lastterm - 2, lastindex - 2, lastindex, []pb.Entry{{Index: lastindex - 1, Term: 4}, {Index: lastindex, Term: 4}},
  222. lastindex, true, lastindex, false,
  223. },
  224. }
  225. for i, tt := range tests {
  226. raftLog := newLog(NewMemoryStorage())
  227. raftLog.append(raftLog.lastIndex(), previousEnts...)
  228. raftLog.committed = commit
  229. func() {
  230. defer func() {
  231. if r := recover(); r != nil {
  232. if tt.wpanic != true {
  233. t.Errorf("%d: panic = %v, want %v", i, true, tt.wpanic)
  234. }
  235. }
  236. }()
  237. glasti, gappend := raftLog.maybeAppend(tt.index, tt.logTerm, tt.committed, tt.ents...)
  238. gcommit := raftLog.committed
  239. if glasti != tt.wlasti {
  240. t.Errorf("#%d: lastindex = %d, want %d", i, glasti, tt.wlasti)
  241. }
  242. if gappend != tt.wappend {
  243. t.Errorf("#%d: append = %v, want %v", i, gappend, tt.wappend)
  244. }
  245. if gcommit != tt.wcommit {
  246. t.Errorf("#%d: committed = %d, want %d", i, gcommit, tt.wcommit)
  247. }
  248. if gappend {
  249. gents := raftLog.slice(raftLog.lastIndex()-uint64(len(tt.ents))+1, raftLog.lastIndex()+1)
  250. if !reflect.DeepEqual(tt.ents, gents) {
  251. t.Errorf("%d: appended entries = %v, want %v", i, gents, tt.ents)
  252. }
  253. }
  254. }()
  255. }
  256. }
  257. // TestCompactionSideEffects ensures that all the log related funcationality works correctly after
  258. // a compaction.
  259. func TestCompactionSideEffects(t *testing.T) {
  260. var i uint64
  261. // Populate the log with 1000 entries; 750 in stable storage and 250 in unstable.
  262. lastIndex := uint64(1000)
  263. unstableIndex := uint64(750)
  264. lastTerm := lastIndex
  265. storage := NewMemoryStorage()
  266. for i = 1; i <= unstableIndex; i++ {
  267. storage.Append([]pb.Entry{{Term: uint64(i), Index: uint64(i)}})
  268. }
  269. raftLog := newLog(storage)
  270. for i = unstableIndex; i < lastIndex; i++ {
  271. raftLog.append(i, pb.Entry{Term: uint64(i + 1), Index: uint64(i + 1)})
  272. }
  273. ok := raftLog.maybeCommit(lastIndex, lastTerm)
  274. if !ok {
  275. t.Fatalf("maybeCommit returned false")
  276. }
  277. raftLog.appliedTo(raftLog.committed)
  278. offset := uint64(500)
  279. storage.Compact(offset, nil, nil)
  280. if raftLog.lastIndex() != lastIndex {
  281. t.Errorf("lastIndex = %d, want %d", raftLog.lastIndex(), lastIndex)
  282. }
  283. for i := offset; i <= raftLog.lastIndex(); i++ {
  284. if raftLog.term(i) != i {
  285. t.Errorf("term(%d) = %d, want %d", i, raftLog.term(i), i)
  286. }
  287. }
  288. for i := offset; i <= raftLog.lastIndex(); i++ {
  289. if !raftLog.matchTerm(i, i) {
  290. t.Errorf("matchTerm(%d) = false, want true", i)
  291. }
  292. }
  293. unstableEnts := raftLog.unstableEntries()
  294. if g := len(unstableEnts); g != 250 {
  295. t.Errorf("len(unstableEntries) = %d, want = %d", g, 250)
  296. }
  297. if unstableEnts[0].Index != 751 {
  298. t.Errorf("Index = %d, want = %d", unstableEnts[0].Index, 751)
  299. }
  300. prev := raftLog.lastIndex()
  301. raftLog.append(raftLog.lastIndex(), pb.Entry{Index: raftLog.lastIndex() + 1, Term: raftLog.lastIndex() + 1})
  302. if raftLog.lastIndex() != prev+1 {
  303. t.Errorf("lastIndex = %d, want = %d", raftLog.lastIndex(), prev+1)
  304. }
  305. ents := raftLog.entries(raftLog.lastIndex())
  306. if len(ents) != 1 {
  307. t.Errorf("len(entries) = %d, want = %d", len(ents), 1)
  308. }
  309. }
  310. func TestNextEnts(t *testing.T) {
  311. snap := pb.Snapshot{
  312. Metadata: pb.SnapshotMetadata{Term: 1, Index: 3},
  313. }
  314. ents := []pb.Entry{
  315. {Term: 1, Index: 4},
  316. {Term: 1, Index: 5},
  317. {Term: 1, Index: 6},
  318. }
  319. tests := []struct {
  320. applied uint64
  321. wents []pb.Entry
  322. }{
  323. {0, ents[:2]},
  324. {3, ents[:2]},
  325. {4, ents[1:2]},
  326. {5, nil},
  327. }
  328. for i, tt := range tests {
  329. storage := NewMemoryStorage()
  330. storage.ApplySnapshot(snap)
  331. raftLog := newLog(storage)
  332. raftLog.append(snap.Metadata.Index, ents...)
  333. raftLog.maybeCommit(5, 1)
  334. raftLog.appliedTo(tt.applied)
  335. ents := raftLog.nextEnts()
  336. if !reflect.DeepEqual(ents, tt.wents) {
  337. t.Errorf("#%d: ents = %+v, want %+v", i, ents, tt.wents)
  338. }
  339. }
  340. }
  341. // TestUnstableEnts ensures unstableEntries returns the unstable part of the
  342. // entries correctly.
  343. func TestUnstableEnts(t *testing.T) {
  344. previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}}
  345. tests := []struct {
  346. unstable uint64
  347. wents []pb.Entry
  348. }{
  349. {3, nil},
  350. {1, previousEnts},
  351. }
  352. for i, tt := range tests {
  353. // append stable entries to storage
  354. storage := NewMemoryStorage()
  355. storage.Append(previousEnts[:tt.unstable-1])
  356. // append unstable entries to raftlog
  357. raftLog := newLog(storage)
  358. raftLog.append(raftLog.lastIndex(), previousEnts[tt.unstable-1:]...)
  359. ents := raftLog.unstableEntries()
  360. if l := len(ents); l > 0 {
  361. raftLog.stableTo(ents[l-1].Index)
  362. }
  363. if !reflect.DeepEqual(ents, tt.wents) {
  364. t.Errorf("#%d: unstableEnts = %+v, want %+v", i, ents, tt.wents)
  365. }
  366. w := previousEnts[len(previousEnts)-1].Index + 1
  367. if g := raftLog.unstable; g != w {
  368. t.Errorf("#%d: unstable = %d, want %d", i, g, w)
  369. }
  370. }
  371. }
  372. func TestCommitTo(t *testing.T) {
  373. previousEnts := []pb.Entry{{Term: 1, Index: 1}, {Term: 2, Index: 2}, {Term: 3, Index: 3}}
  374. commit := uint64(2)
  375. tests := []struct {
  376. commit uint64
  377. wcommit uint64
  378. wpanic bool
  379. }{
  380. {3, 3, false},
  381. {1, 2, false}, // never decrease
  382. {4, 0, true}, // commit out of range -> panic
  383. }
  384. for i, tt := range tests {
  385. func() {
  386. defer func() {
  387. if r := recover(); r != nil {
  388. if tt.wpanic != true {
  389. t.Errorf("%d: panic = %v, want %v", i, true, tt.wpanic)
  390. }
  391. }
  392. }()
  393. raftLog := newLog(NewMemoryStorage())
  394. raftLog.append(0, previousEnts...)
  395. raftLog.committed = commit
  396. raftLog.commitTo(tt.commit)
  397. if raftLog.committed != tt.wcommit {
  398. t.Errorf("#%d: committed = %d, want %d", i, raftLog.committed, tt.wcommit)
  399. }
  400. }()
  401. }
  402. }
  403. func TestStableTo(t *testing.T) {
  404. tests := []struct {
  405. stable uint64
  406. wunstable uint64
  407. }{
  408. {1, 2},
  409. {2, 3},
  410. }
  411. for i, tt := range tests {
  412. raftLog := newLog(NewMemoryStorage())
  413. raftLog.append(0, []pb.Entry{{}, {}}...)
  414. raftLog.stableTo(tt.stable)
  415. if raftLog.unstable != tt.wunstable {
  416. t.Errorf("#%d: unstable = %d, want %d", i, raftLog.unstable, tt.wunstable)
  417. }
  418. }
  419. }
  420. //TestCompaction ensures that the number of log entries is correct after compactions.
  421. func TestCompaction(t *testing.T) {
  422. tests := []struct {
  423. lastIndex uint64
  424. compact []uint64
  425. wleft []int
  426. wallow bool
  427. }{
  428. // out of upper bound
  429. {1000, []uint64{1001}, []int{-1}, false},
  430. {1000, []uint64{300, 500, 800, 900}, []int{700, 500, 200, 100}, true},
  431. // out of lower bound
  432. {1000, []uint64{300, 299}, []int{700, -1}, false},
  433. }
  434. for i, tt := range tests {
  435. func() {
  436. defer func() {
  437. if r := recover(); r != nil {
  438. if tt.wallow == true {
  439. t.Errorf("%d: allow = %v, want %v: %v", i, false, true, r)
  440. }
  441. }
  442. }()
  443. storage := NewMemoryStorage()
  444. for i := uint64(1); i <= tt.lastIndex; i++ {
  445. storage.Append([]pb.Entry{{Index: i}})
  446. }
  447. raftLog := newLog(storage)
  448. raftLog.maybeCommit(tt.lastIndex, 0)
  449. raftLog.appliedTo(raftLog.committed)
  450. for j := 0; j < len(tt.compact); j++ {
  451. err := storage.Compact(tt.compact[j], nil, nil)
  452. if err != nil {
  453. if tt.wallow {
  454. t.Errorf("#%d.%d allow = %t, want %t", i, j, false, tt.wallow)
  455. }
  456. continue
  457. }
  458. if len(raftLog.allEntries()) != tt.wleft[j] {
  459. t.Errorf("#%d.%d len = %d, want %d", i, j, len(raftLog.allEntries()), tt.wleft[j])
  460. }
  461. }
  462. }()
  463. }
  464. }
  465. func TestLogRestore(t *testing.T) {
  466. index := uint64(1000)
  467. term := uint64(1000)
  468. snap := pb.SnapshotMetadata{Index: index, Term: term}
  469. storage := NewMemoryStorage()
  470. storage.ApplySnapshot(pb.Snapshot{Metadata: snap})
  471. raftLog := newLog(storage)
  472. if len(raftLog.allEntries()) != 0 {
  473. t.Errorf("len = %d, want 0", len(raftLog.allEntries()))
  474. }
  475. if raftLog.firstIndex() != index+1 {
  476. t.Errorf("firstIndex = %d, want %d", raftLog.firstIndex(), index+1)
  477. }
  478. if raftLog.committed != index {
  479. t.Errorf("comitted = %d, want %d", raftLog.committed, index)
  480. }
  481. if raftLog.unstable != index+1 {
  482. t.Errorf("unstable = %d, want %d", raftLog.unstable, index+1)
  483. }
  484. if raftLog.term(index) != term {
  485. t.Errorf("term = %d, want %d", raftLog.term(index), term)
  486. }
  487. }
  488. func TestIsOutOfBounds(t *testing.T) {
  489. offset := uint64(100)
  490. num := uint64(100)
  491. storage := NewMemoryStorage()
  492. storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
  493. l := newLog(storage)
  494. l.append(offset, make([]pb.Entry, num)...)
  495. tests := []struct {
  496. index uint64
  497. w bool
  498. }{
  499. {offset - 1, true},
  500. {offset, true},
  501. {offset + num/2, false},
  502. {offset + num, false},
  503. {offset + num + 1, true},
  504. }
  505. for i, tt := range tests {
  506. g := l.isOutOfBounds(tt.index)
  507. if g != tt.w {
  508. t.Errorf("#%d: isOutOfBounds = %v, want %v", i, g, tt.w)
  509. }
  510. }
  511. }
  512. func TestTerm(t *testing.T) {
  513. var i uint64
  514. offset := uint64(100)
  515. num := uint64(100)
  516. storage := NewMemoryStorage()
  517. storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
  518. l := newLog(storage)
  519. for i = 1; i < num; i++ {
  520. l.append(offset+i-1, pb.Entry{Index: i, Term: i})
  521. }
  522. tests := []struct {
  523. index uint64
  524. w uint64
  525. }{
  526. {offset - 1, 0},
  527. {offset, 0},
  528. {offset + num/2, num / 2},
  529. {offset + num - 1, num - 1},
  530. {offset + num, 0},
  531. }
  532. for i, tt := range tests {
  533. term := l.term(tt.index)
  534. if !reflect.DeepEqual(term, tt.w) {
  535. t.Errorf("#%d: at = %d, want %d", i, term, tt.w)
  536. }
  537. }
  538. }
  539. func TestSlice(t *testing.T) {
  540. var i uint64
  541. offset := uint64(100)
  542. num := uint64(100)
  543. storage := NewMemoryStorage()
  544. storage.ApplySnapshot(pb.Snapshot{Metadata: pb.SnapshotMetadata{Index: offset}})
  545. l := newLog(storage)
  546. for i = 1; i < num; i++ {
  547. l.append(offset+i-1, pb.Entry{Index: i, Term: i})
  548. }
  549. tests := []struct {
  550. from uint64
  551. to uint64
  552. w []pb.Entry
  553. }{
  554. {offset - 1, offset + 1, nil},
  555. {offset, offset + 1, nil},
  556. {offset + num/2, offset + num/2 + 1, []pb.Entry{{Index: num / 2, Term: num / 2}}},
  557. {offset + num - 1, offset + num, []pb.Entry{{Index: num - 1, Term: num - 1}}},
  558. {offset + num, offset + num + 1, nil},
  559. {offset + num/2, offset + num/2, nil},
  560. {offset + num/2, offset + num/2 - 1, nil},
  561. }
  562. for i, tt := range tests {
  563. g := l.slice(tt.from, tt.to)
  564. if !reflect.DeepEqual(g, tt.w) {
  565. t.Errorf("#%d: from %d to %d = %v, want %v", i, tt.from, tt.to, g, tt.w)
  566. }
  567. }
  568. }