raft_flow_control_test.go 4.8 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156
  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. "testing"
  17. pb "go.etcd.io/etcd/raft/raftpb"
  18. )
  19. // TestMsgAppFlowControlFull ensures:
  20. // 1. msgApp can fill the sending window until full
  21. // 2. when the window is full, no more msgApp can be sent.
  22. func TestMsgAppFlowControlFull(t *testing.T) {
  23. r := newTestRaft(1, []uint64{1, 2}, 5, 1, NewMemoryStorage())
  24. r.becomeCandidate()
  25. r.becomeLeader()
  26. pr2 := r.prs.Progress[2]
  27. // force the progress to be in replicate state
  28. pr2.BecomeReplicate()
  29. // fill in the inflights window
  30. for i := 0; i < r.prs.MaxInflight; i++ {
  31. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  32. ms := r.readMessages()
  33. if len(ms) != 1 {
  34. t.Fatalf("#%d: len(ms) = %d, want 1", i, len(ms))
  35. }
  36. }
  37. // ensure 1
  38. if !pr2.Inflights.Full() {
  39. t.Fatalf("inflights.full = %t, want %t", pr2.Inflights.Full(), true)
  40. }
  41. // ensure 2
  42. for i := 0; i < 10; i++ {
  43. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  44. ms := r.readMessages()
  45. if len(ms) != 0 {
  46. t.Fatalf("#%d: len(ms) = %d, want 0", i, len(ms))
  47. }
  48. }
  49. }
  50. // TestMsgAppFlowControlMoveForward ensures msgAppResp can move
  51. // forward the sending window correctly:
  52. // 1. valid msgAppResp.index moves the windows to pass all smaller or equal index.
  53. // 2. out-of-dated msgAppResp has no effect on the sliding window.
  54. func TestMsgAppFlowControlMoveForward(t *testing.T) {
  55. r := newTestRaft(1, []uint64{1, 2}, 5, 1, NewMemoryStorage())
  56. r.becomeCandidate()
  57. r.becomeLeader()
  58. pr2 := r.prs.Progress[2]
  59. // force the progress to be in replicate state
  60. pr2.BecomeReplicate()
  61. // fill in the inflights window
  62. for i := 0; i < r.prs.MaxInflight; i++ {
  63. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  64. r.readMessages()
  65. }
  66. // 1 is noop, 2 is the first proposal we just sent.
  67. // so we start with 2.
  68. for tt := 2; tt < r.prs.MaxInflight; tt++ {
  69. // move forward the window
  70. r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: uint64(tt)})
  71. r.readMessages()
  72. // fill in the inflights window again
  73. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  74. ms := r.readMessages()
  75. if len(ms) != 1 {
  76. t.Fatalf("#%d: len(ms) = %d, want 1", tt, len(ms))
  77. }
  78. // ensure 1
  79. if !pr2.Inflights.Full() {
  80. t.Fatalf("inflights.full = %t, want %t", pr2.Inflights.Full(), true)
  81. }
  82. // ensure 2
  83. for i := 0; i < tt; i++ {
  84. r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: uint64(i)})
  85. if !pr2.Inflights.Full() {
  86. t.Fatalf("#%d: inflights.full = %t, want %t", tt, pr2.Inflights.Full(), true)
  87. }
  88. }
  89. }
  90. }
  91. // TestMsgAppFlowControlRecvHeartbeat ensures a heartbeat response
  92. // frees one slot if the window is full.
  93. func TestMsgAppFlowControlRecvHeartbeat(t *testing.T) {
  94. r := newTestRaft(1, []uint64{1, 2}, 5, 1, NewMemoryStorage())
  95. r.becomeCandidate()
  96. r.becomeLeader()
  97. pr2 := r.prs.Progress[2]
  98. // force the progress to be in replicate state
  99. pr2.BecomeReplicate()
  100. // fill in the inflights window
  101. for i := 0; i < r.prs.MaxInflight; i++ {
  102. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  103. r.readMessages()
  104. }
  105. for tt := 1; tt < 5; tt++ {
  106. if !pr2.Inflights.Full() {
  107. t.Fatalf("#%d: inflights.full = %t, want %t", tt, pr2.Inflights.Full(), true)
  108. }
  109. // recv tt msgHeartbeatResp and expect one free slot
  110. for i := 0; i < tt; i++ {
  111. r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
  112. r.readMessages()
  113. if pr2.Inflights.Full() {
  114. t.Fatalf("#%d.%d: inflights.full = %t, want %t", tt, i, pr2.Inflights.Full(), false)
  115. }
  116. }
  117. // one slot
  118. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  119. ms := r.readMessages()
  120. if len(ms) != 1 {
  121. t.Fatalf("#%d: free slot = 0, want 1", tt)
  122. }
  123. // and just one slot
  124. for i := 0; i < 10; i++ {
  125. r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
  126. ms1 := r.readMessages()
  127. if len(ms1) != 0 {
  128. t.Fatalf("#%d.%d: len(ms) = %d, want 0", tt, i, len(ms1))
  129. }
  130. }
  131. // clear all pending messages.
  132. r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
  133. r.readMessages()
  134. }
  135. }