rawnode_test.go 17 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618
  1. // Copyright 2015 The etcd Authors
  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 raft
  15. import (
  16. "bytes"
  17. "fmt"
  18. "reflect"
  19. "testing"
  20. "go.etcd.io/etcd/raft/raftpb"
  21. )
  22. // TestRawNodeStep ensures that RawNode.Step ignore local message.
  23. func TestRawNodeStep(t *testing.T) {
  24. for i, msgn := range raftpb.MessageType_name {
  25. s := NewMemoryStorage()
  26. rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, s), []Peer{{ID: 1}})
  27. if err != nil {
  28. t.Fatal(err)
  29. }
  30. msgt := raftpb.MessageType(i)
  31. err = rawNode.Step(raftpb.Message{Type: msgt})
  32. // LocalMsg should be ignored.
  33. if IsLocalMsg(msgt) {
  34. if err != ErrStepLocalMsg {
  35. t.Errorf("%d: step should ignore %s", msgt, msgn)
  36. }
  37. }
  38. }
  39. }
  40. // TestNodeStepUnblock from node_test.go has no equivalent in rawNode because there is
  41. // no goroutine in RawNode.
  42. // TestRawNodeProposeAndConfChange ensures that RawNode.Propose and RawNode.ProposeConfChange
  43. // send the given proposal and ConfChange to the underlying raft.
  44. func TestRawNodeProposeAndConfChange(t *testing.T) {
  45. s := NewMemoryStorage()
  46. var err error
  47. rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, s), []Peer{{ID: 1}})
  48. if err != nil {
  49. t.Fatal(err)
  50. }
  51. rd := rawNode.Ready()
  52. s.Append(rd.Entries)
  53. rawNode.Advance(rd)
  54. if d := rawNode.Ready(); d.MustSync || !IsEmptyHardState(d.HardState) || len(d.Entries) > 0 {
  55. t.Fatalf("expected empty hard state with must-sync=false: %#v", d)
  56. }
  57. rawNode.Campaign()
  58. proposed := false
  59. var (
  60. lastIndex uint64
  61. ccdata []byte
  62. )
  63. for {
  64. rd = rawNode.Ready()
  65. s.Append(rd.Entries)
  66. // Once we are the leader, propose a command and a ConfChange.
  67. if !proposed && rd.SoftState.Lead == rawNode.raft.id {
  68. rawNode.Propose([]byte("somedata"))
  69. cc := raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1}
  70. ccdata, err = cc.Marshal()
  71. if err != nil {
  72. t.Fatal(err)
  73. }
  74. rawNode.ProposeConfChange(cc)
  75. proposed = true
  76. }
  77. rawNode.Advance(rd)
  78. // Exit when we have four entries: one ConfChange, one no-op for the election,
  79. // our proposed command and proposed ConfChange.
  80. lastIndex, err = s.LastIndex()
  81. if err != nil {
  82. t.Fatal(err)
  83. }
  84. if lastIndex >= 4 {
  85. break
  86. }
  87. }
  88. entries, err := s.Entries(lastIndex-1, lastIndex+1, noLimit)
  89. if err != nil {
  90. t.Fatal(err)
  91. }
  92. if len(entries) != 2 {
  93. t.Fatalf("len(entries) = %d, want %d", len(entries), 2)
  94. }
  95. if !bytes.Equal(entries[0].Data, []byte("somedata")) {
  96. t.Errorf("entries[0].Data = %v, want %v", entries[0].Data, []byte("somedata"))
  97. }
  98. if entries[1].Type != raftpb.EntryConfChange {
  99. t.Fatalf("type = %v, want %v", entries[1].Type, raftpb.EntryConfChange)
  100. }
  101. if !bytes.Equal(entries[1].Data, ccdata) {
  102. t.Errorf("data = %v, want %v", entries[1].Data, ccdata)
  103. }
  104. }
  105. // TestRawNodeProposeAddDuplicateNode ensures that two proposes to add the same node should
  106. // not affect the later propose to add new node.
  107. func TestRawNodeProposeAddDuplicateNode(t *testing.T) {
  108. s := NewMemoryStorage()
  109. rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, s), []Peer{{ID: 1}})
  110. if err != nil {
  111. t.Fatal(err)
  112. }
  113. rd := rawNode.Ready()
  114. s.Append(rd.Entries)
  115. rawNode.Advance(rd)
  116. rawNode.Campaign()
  117. for {
  118. rd = rawNode.Ready()
  119. s.Append(rd.Entries)
  120. if rd.SoftState.Lead == rawNode.raft.id {
  121. rawNode.Advance(rd)
  122. break
  123. }
  124. rawNode.Advance(rd)
  125. }
  126. proposeConfChangeAndApply := func(cc raftpb.ConfChange) {
  127. rawNode.ProposeConfChange(cc)
  128. rd = rawNode.Ready()
  129. s.Append(rd.Entries)
  130. for _, entry := range rd.CommittedEntries {
  131. if entry.Type == raftpb.EntryConfChange {
  132. var cc raftpb.ConfChange
  133. cc.Unmarshal(entry.Data)
  134. rawNode.ApplyConfChange(cc)
  135. }
  136. }
  137. rawNode.Advance(rd)
  138. }
  139. cc1 := raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1}
  140. ccdata1, err := cc1.Marshal()
  141. if err != nil {
  142. t.Fatal(err)
  143. }
  144. proposeConfChangeAndApply(cc1)
  145. // try to add the same node again
  146. proposeConfChangeAndApply(cc1)
  147. // the new node join should be ok
  148. cc2 := raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 2}
  149. ccdata2, err := cc2.Marshal()
  150. if err != nil {
  151. t.Fatal(err)
  152. }
  153. proposeConfChangeAndApply(cc2)
  154. lastIndex, err := s.LastIndex()
  155. if err != nil {
  156. t.Fatal(err)
  157. }
  158. // the last three entries should be: ConfChange cc1, cc1, cc2
  159. entries, err := s.Entries(lastIndex-2, lastIndex+1, noLimit)
  160. if err != nil {
  161. t.Fatal(err)
  162. }
  163. if len(entries) != 3 {
  164. t.Fatalf("len(entries) = %d, want %d", len(entries), 3)
  165. }
  166. if !bytes.Equal(entries[0].Data, ccdata1) {
  167. t.Errorf("entries[0].Data = %v, want %v", entries[0].Data, ccdata1)
  168. }
  169. if !bytes.Equal(entries[2].Data, ccdata2) {
  170. t.Errorf("entries[2].Data = %v, want %v", entries[2].Data, ccdata2)
  171. }
  172. }
  173. // TestRawNodeReadIndex ensures that Rawnode.ReadIndex sends the MsgReadIndex message
  174. // to the underlying raft. It also ensures that ReadState can be read out.
  175. func TestRawNodeReadIndex(t *testing.T) {
  176. msgs := []raftpb.Message{}
  177. appendStep := func(r *raft, m raftpb.Message) error {
  178. msgs = append(msgs, m)
  179. return nil
  180. }
  181. wrs := []ReadState{{Index: uint64(1), RequestCtx: []byte("somedata")}}
  182. s := NewMemoryStorage()
  183. c := newTestConfig(1, nil, 10, 1, s)
  184. rawNode, err := NewRawNode(c, []Peer{{ID: 1}})
  185. if err != nil {
  186. t.Fatal(err)
  187. }
  188. rawNode.raft.readStates = wrs
  189. // ensure the ReadStates can be read out
  190. hasReady := rawNode.HasReady()
  191. if !hasReady {
  192. t.Errorf("HasReady() returns %t, want %t", hasReady, true)
  193. }
  194. rd := rawNode.Ready()
  195. if !reflect.DeepEqual(rd.ReadStates, wrs) {
  196. t.Errorf("ReadStates = %d, want %d", rd.ReadStates, wrs)
  197. }
  198. s.Append(rd.Entries)
  199. rawNode.Advance(rd)
  200. // ensure raft.readStates is reset after advance
  201. if rawNode.raft.readStates != nil {
  202. t.Errorf("readStates = %v, want %v", rawNode.raft.readStates, nil)
  203. }
  204. wrequestCtx := []byte("somedata2")
  205. rawNode.Campaign()
  206. for {
  207. rd = rawNode.Ready()
  208. s.Append(rd.Entries)
  209. if rd.SoftState.Lead == rawNode.raft.id {
  210. rawNode.Advance(rd)
  211. // Once we are the leader, issue a ReadIndex request
  212. rawNode.raft.step = appendStep
  213. rawNode.ReadIndex(wrequestCtx)
  214. break
  215. }
  216. rawNode.Advance(rd)
  217. }
  218. // ensure that MsgReadIndex message is sent to the underlying raft
  219. if len(msgs) != 1 {
  220. t.Fatalf("len(msgs) = %d, want %d", len(msgs), 1)
  221. }
  222. if msgs[0].Type != raftpb.MsgReadIndex {
  223. t.Errorf("msg type = %d, want %d", msgs[0].Type, raftpb.MsgReadIndex)
  224. }
  225. if !bytes.Equal(msgs[0].Entries[0].Data, wrequestCtx) {
  226. t.Errorf("data = %v, want %v", msgs[0].Entries[0].Data, wrequestCtx)
  227. }
  228. }
  229. // TestBlockProposal from node_test.go has no equivalent in rawNode because there is
  230. // no leader check in RawNode.
  231. // TestNodeTick from node_test.go has no equivalent in rawNode because
  232. // it reaches into the raft object which is not exposed.
  233. // TestNodeStop from node_test.go has no equivalent in rawNode because there is
  234. // no goroutine in RawNode.
  235. // TestRawNodeStart ensures that a node can be started correctly. The node should
  236. // start with correct configuration change entries, and can accept and commit
  237. // proposals.
  238. func TestRawNodeStart(t *testing.T) {
  239. cc := raftpb.ConfChange{Type: raftpb.ConfChangeAddNode, NodeID: 1}
  240. ccdata, err := cc.Marshal()
  241. if err != nil {
  242. t.Fatalf("unexpected marshal error: %v", err)
  243. }
  244. wants := []Ready{
  245. {
  246. HardState: raftpb.HardState{Term: 1, Commit: 1, Vote: 0},
  247. Entries: []raftpb.Entry{
  248. {Type: raftpb.EntryConfChange, Term: 1, Index: 1, Data: ccdata},
  249. },
  250. CommittedEntries: []raftpb.Entry{
  251. {Type: raftpb.EntryConfChange, Term: 1, Index: 1, Data: ccdata},
  252. },
  253. MustSync: true,
  254. },
  255. {
  256. HardState: raftpb.HardState{Term: 2, Commit: 3, Vote: 1},
  257. Entries: []raftpb.Entry{{Term: 2, Index: 3, Data: []byte("foo")}},
  258. CommittedEntries: []raftpb.Entry{{Term: 2, Index: 3, Data: []byte("foo")}},
  259. MustSync: true,
  260. },
  261. }
  262. storage := NewMemoryStorage()
  263. rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, storage), []Peer{{ID: 1}})
  264. if err != nil {
  265. t.Fatal(err)
  266. }
  267. rd := rawNode.Ready()
  268. t.Logf("rd %v", rd)
  269. if !reflect.DeepEqual(rd, wants[0]) {
  270. t.Fatalf("#%d: g = %+v,\n w %+v", 1, rd, wants[0])
  271. } else {
  272. storage.Append(rd.Entries)
  273. rawNode.Advance(rd)
  274. }
  275. storage.Append(rd.Entries)
  276. rawNode.Advance(rd)
  277. rawNode.Campaign()
  278. rd = rawNode.Ready()
  279. storage.Append(rd.Entries)
  280. rawNode.Advance(rd)
  281. rawNode.Propose([]byte("foo"))
  282. if rd = rawNode.Ready(); !reflect.DeepEqual(rd, wants[1]) {
  283. t.Errorf("#%d: g = %+v,\n w %+v", 2, rd, wants[1])
  284. } else {
  285. storage.Append(rd.Entries)
  286. rawNode.Advance(rd)
  287. }
  288. if rawNode.HasReady() {
  289. t.Errorf("unexpected Ready: %+v", rawNode.Ready())
  290. }
  291. }
  292. func TestRawNodeRestart(t *testing.T) {
  293. entries := []raftpb.Entry{
  294. {Term: 1, Index: 1},
  295. {Term: 1, Index: 2, Data: []byte("foo")},
  296. }
  297. st := raftpb.HardState{Term: 1, Commit: 1}
  298. want := Ready{
  299. HardState: emptyState,
  300. // commit up to commit index in st
  301. CommittedEntries: entries[:st.Commit],
  302. MustSync: false,
  303. }
  304. storage := NewMemoryStorage()
  305. storage.SetHardState(st)
  306. storage.Append(entries)
  307. rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, storage), nil)
  308. if err != nil {
  309. t.Fatal(err)
  310. }
  311. rd := rawNode.Ready()
  312. if !reflect.DeepEqual(rd, want) {
  313. t.Errorf("g = %+v,\n w %+v", rd, want)
  314. }
  315. rawNode.Advance(rd)
  316. if rawNode.HasReady() {
  317. t.Errorf("unexpected Ready: %+v", rawNode.Ready())
  318. }
  319. }
  320. func TestRawNodeRestartFromSnapshot(t *testing.T) {
  321. snap := raftpb.Snapshot{
  322. Metadata: raftpb.SnapshotMetadata{
  323. ConfState: raftpb.ConfState{Nodes: []uint64{1, 2}},
  324. Index: 2,
  325. Term: 1,
  326. },
  327. }
  328. entries := []raftpb.Entry{
  329. {Term: 1, Index: 3, Data: []byte("foo")},
  330. }
  331. st := raftpb.HardState{Term: 1, Commit: 3}
  332. want := Ready{
  333. HardState: emptyState,
  334. // commit up to commit index in st
  335. CommittedEntries: entries,
  336. MustSync: false,
  337. }
  338. s := NewMemoryStorage()
  339. s.SetHardState(st)
  340. s.ApplySnapshot(snap)
  341. s.Append(entries)
  342. rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, s), nil)
  343. if err != nil {
  344. t.Fatal(err)
  345. }
  346. if rd := rawNode.Ready(); !reflect.DeepEqual(rd, want) {
  347. t.Errorf("g = %+v,\n w %+v", rd, want)
  348. } else {
  349. rawNode.Advance(rd)
  350. }
  351. if rawNode.HasReady() {
  352. t.Errorf("unexpected Ready: %+v", rawNode.HasReady())
  353. }
  354. }
  355. // TestNodeAdvance from node_test.go has no equivalent in rawNode because there is
  356. // no dependency check between Ready() and Advance()
  357. func TestRawNodeStatus(t *testing.T) {
  358. storage := NewMemoryStorage()
  359. rawNode, err := NewRawNode(newTestConfig(1, nil, 10, 1, storage), []Peer{{ID: 1}})
  360. if err != nil {
  361. t.Fatal(err)
  362. }
  363. status := rawNode.Status()
  364. if status == nil {
  365. t.Errorf("expected status struct, got nil")
  366. }
  367. }
  368. // TestRawNodeCommitPaginationAfterRestart is the RawNode version of
  369. // TestNodeCommitPaginationAfterRestart. The anomaly here was even worse as the
  370. // Raft group would forget to apply entries:
  371. //
  372. // - node learns that index 11 is committed
  373. // - nextEnts returns index 1..10 in CommittedEntries (but index 10 already
  374. // exceeds maxBytes), which isn't noticed internally by Raft
  375. // - Commit index gets bumped to 10
  376. // - the node persists the HardState, but crashes before applying the entries
  377. // - upon restart, the storage returns the same entries, but `slice` takes a
  378. // different code path and removes the last entry.
  379. // - Raft does not emit a HardState, but when the app calls Advance(), it bumps
  380. // its internal applied index cursor to 10 (when it should be 9)
  381. // - the next Ready asks the app to apply index 11 (omitting index 10), losing a
  382. // write.
  383. func TestRawNodeCommitPaginationAfterRestart(t *testing.T) {
  384. s := &ignoreSizeHintMemStorage{
  385. MemoryStorage: NewMemoryStorage(),
  386. }
  387. persistedHardState := raftpb.HardState{
  388. Term: 1,
  389. Vote: 1,
  390. Commit: 10,
  391. }
  392. s.hardState = persistedHardState
  393. s.ents = make([]raftpb.Entry, 10)
  394. var size uint64
  395. for i := range s.ents {
  396. ent := raftpb.Entry{
  397. Term: 1,
  398. Index: uint64(i + 1),
  399. Type: raftpb.EntryNormal,
  400. Data: []byte("a"),
  401. }
  402. s.ents[i] = ent
  403. size += uint64(ent.Size())
  404. }
  405. cfg := newTestConfig(1, []uint64{1}, 10, 1, s)
  406. // Set a MaxSizePerMsg that would suggest to Raft that the last committed entry should
  407. // not be included in the initial rd.CommittedEntries. However, our storage will ignore
  408. // this and *will* return it (which is how the Commit index ended up being 10 initially).
  409. cfg.MaxSizePerMsg = size - uint64(s.ents[len(s.ents)-1].Size()) - 1
  410. s.ents = append(s.ents, raftpb.Entry{
  411. Term: 1,
  412. Index: uint64(11),
  413. Type: raftpb.EntryNormal,
  414. Data: []byte("boom"),
  415. })
  416. rawNode, err := NewRawNode(cfg, []Peer{{ID: 1}})
  417. if err != nil {
  418. t.Fatal(err)
  419. }
  420. for highestApplied := uint64(0); highestApplied != 11; {
  421. rd := rawNode.Ready()
  422. n := len(rd.CommittedEntries)
  423. if n == 0 {
  424. t.Fatalf("stopped applying entries at index %d", highestApplied)
  425. }
  426. if next := rd.CommittedEntries[0].Index; highestApplied != 0 && highestApplied+1 != next {
  427. t.Fatalf("attempting to apply index %d after index %d, leaving a gap", next, highestApplied)
  428. }
  429. highestApplied = rd.CommittedEntries[n-1].Index
  430. rawNode.Advance(rd)
  431. rawNode.Step(raftpb.Message{
  432. Type: raftpb.MsgHeartbeat,
  433. To: 1,
  434. From: 1, // illegal, but we get away with it
  435. Term: 1,
  436. Commit: 11,
  437. })
  438. }
  439. }
  440. // TestRawNodeBoundedLogGrowthWithPartition tests a scenario where a leader is
  441. // partitioned from a quorum of nodes. It verifies that the leader's log is
  442. // protected from unbounded growth even as new entries continue to be proposed.
  443. // This protection is provided by the MaxUncommittedEntriesSize configuration.
  444. func TestRawNodeBoundedLogGrowthWithPartition(t *testing.T) {
  445. const maxEntries = 16
  446. data := []byte("testdata")
  447. testEntry := raftpb.Entry{Data: data}
  448. maxEntrySize := uint64(maxEntries * PayloadSize(testEntry))
  449. s := NewMemoryStorage()
  450. cfg := newTestConfig(1, []uint64{1}, 10, 1, s)
  451. cfg.MaxUncommittedEntriesSize = maxEntrySize
  452. rawNode, err := NewRawNode(cfg, []Peer{{ID: 1}})
  453. if err != nil {
  454. t.Fatal(err)
  455. }
  456. rd := rawNode.Ready()
  457. s.Append(rd.Entries)
  458. rawNode.Advance(rd)
  459. // Become the leader.
  460. rawNode.Campaign()
  461. for {
  462. rd = rawNode.Ready()
  463. s.Append(rd.Entries)
  464. if rd.SoftState.Lead == rawNode.raft.id {
  465. rawNode.Advance(rd)
  466. break
  467. }
  468. rawNode.Advance(rd)
  469. }
  470. // Simulate a network partition while we make our proposals by never
  471. // committing anything. These proposals should not cause the leader's
  472. // log to grow indefinitely.
  473. for i := 0; i < 1024; i++ {
  474. rawNode.Propose(data)
  475. }
  476. // Check the size of leader's uncommitted log tail. It should not exceed the
  477. // MaxUncommittedEntriesSize limit.
  478. checkUncommitted := func(exp uint64) {
  479. t.Helper()
  480. if a := rawNode.raft.uncommittedSize; exp != a {
  481. t.Fatalf("expected %d uncommitted entry bytes, found %d", exp, a)
  482. }
  483. }
  484. checkUncommitted(maxEntrySize)
  485. // Recover from the partition. The uncommitted tail of the Raft log should
  486. // disappear as entries are committed.
  487. rd = rawNode.Ready()
  488. if len(rd.CommittedEntries) != maxEntries {
  489. t.Fatalf("expected %d entries, got %d", maxEntries, len(rd.CommittedEntries))
  490. }
  491. s.Append(rd.Entries)
  492. rawNode.Advance(rd)
  493. checkUncommitted(0)
  494. }
  495. func BenchmarkStatusProgress(b *testing.B) {
  496. setup := func(members int) *RawNode {
  497. peers := make([]uint64, members)
  498. for i := range peers {
  499. peers[i] = uint64(i + 1)
  500. }
  501. cfg := newTestConfig(1, peers, 3, 1, NewMemoryStorage())
  502. cfg.Logger = discardLogger
  503. r := newRaft(cfg)
  504. r.becomeFollower(1, 1)
  505. r.becomeCandidate()
  506. r.becomeLeader()
  507. return &RawNode{raft: r}
  508. }
  509. for _, members := range []int{1, 3, 5, 100} {
  510. b.Run(fmt.Sprintf("members=%d", members), func(b *testing.B) {
  511. // NB: call getStatus through rn.Status because that incurs an additional
  512. // allocation.
  513. rn := setup(members)
  514. b.Run("Status", func(b *testing.B) {
  515. b.ReportAllocs()
  516. for i := 0; i < b.N; i++ {
  517. _ = rn.Status()
  518. }
  519. })
  520. b.Run("Status-example", func(b *testing.B) {
  521. b.ReportAllocs()
  522. for i := 0; i < b.N; i++ {
  523. s := rn.Status()
  524. var n uint64
  525. for _, pr := range s.Progress {
  526. n += pr.Match
  527. }
  528. _ = n
  529. }
  530. })
  531. b.Run("StatusWithoutProgress", func(b *testing.B) {
  532. b.ReportAllocs()
  533. for i := 0; i < b.N; i++ {
  534. _ = rn.StatusWithoutProgress()
  535. }
  536. })
  537. b.Run("WithProgress", func(b *testing.B) {
  538. b.ReportAllocs()
  539. visit := func(uint64, ProgressType, Progress) {}
  540. for i := 0; i < b.N; i++ {
  541. rn.WithProgress(visit)
  542. }
  543. })
  544. b.Run("WithProgress-example", func(b *testing.B) {
  545. b.ReportAllocs()
  546. for i := 0; i < b.N; i++ {
  547. var n uint64
  548. visit := func(_ uint64, _ ProgressType, pr Progress) {
  549. n += pr.Match
  550. }
  551. rn.WithProgress(visit)
  552. _ = n
  553. }
  554. })
  555. })
  556. }
  557. }