|
|
@@ -26,6 +26,19 @@ import (
|
|
|
"github.com/coreos/etcd/raft/raftpb"
|
|
|
)
|
|
|
|
|
|
+// readyWithTimeout selects from n.Ready() with a 1-second timeout. It
|
|
|
+// panics on timeout, which is better than the indefinite wait that
|
|
|
+// would occur if this channel were read without being wrapped in a
|
|
|
+// select.
|
|
|
+func readyWithTimeout(n Node) Ready {
|
|
|
+ select {
|
|
|
+ case rd := <-n.Ready():
|
|
|
+ return rd
|
|
|
+ case <-time.After(time.Second):
|
|
|
+ panic("timed out waiting for ready")
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
// TestNodeStep ensures that node.Step sends msgProp to propc chan
|
|
|
// and other kinds of messages to recvc chan.
|
|
|
func TestNodeStep(t *testing.T) {
|
|
|
@@ -831,3 +844,85 @@ func TestNodeProposeAddLearnerNode(t *testing.T) {
|
|
|
close(stop)
|
|
|
<-done
|
|
|
}
|
|
|
+
|
|
|
+func TestAppendPagination(t *testing.T) {
|
|
|
+ const maxSizePerMsg = 2048
|
|
|
+ n := newNetworkWithConfig(func(c *Config) {
|
|
|
+ c.MaxSizePerMsg = maxSizePerMsg
|
|
|
+ }, nil, nil, nil)
|
|
|
+
|
|
|
+ seenFullMessage := false
|
|
|
+ // Inspect all messages to see that we never exceed the limit, but
|
|
|
+ // we do see messages of larger than half the limit.
|
|
|
+ n.msgHook = func(m raftpb.Message) bool {
|
|
|
+ if m.Type == raftpb.MsgApp {
|
|
|
+ size := 0
|
|
|
+ for _, e := range m.Entries {
|
|
|
+ size += len(e.Data)
|
|
|
+ }
|
|
|
+ if size > maxSizePerMsg {
|
|
|
+ t.Errorf("sent MsgApp that is too large: %d bytes", size)
|
|
|
+ }
|
|
|
+ if size > maxSizePerMsg/2 {
|
|
|
+ seenFullMessage = true
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return true
|
|
|
+ }
|
|
|
+
|
|
|
+ n.send(raftpb.Message{From: 1, To: 1, Type: raftpb.MsgHup})
|
|
|
+
|
|
|
+ // Partition the network while we make our proposals. This forces
|
|
|
+ // the entries to be batched into larger messages.
|
|
|
+ n.isolate(1)
|
|
|
+ blob := []byte(strings.Repeat("a", 1000))
|
|
|
+ for i := 0; i < 5; i++ {
|
|
|
+ n.send(raftpb.Message{From: 1, To: 1, Type: raftpb.MsgProp, Entries: []raftpb.Entry{{Data: blob}}})
|
|
|
+ }
|
|
|
+ n.recover()
|
|
|
+
|
|
|
+ // After the partition recovers, tick the clock to wake everything
|
|
|
+ // back up and send the messages.
|
|
|
+ n.send(raftpb.Message{From: 1, To: 1, Type: raftpb.MsgBeat})
|
|
|
+ if !seenFullMessage {
|
|
|
+ t.Error("didn't see any messages more than half the max size; something is wrong with this test")
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
+func TestCommitPagination(t *testing.T) {
|
|
|
+ s := NewMemoryStorage()
|
|
|
+ cfg := newTestConfig(1, []uint64{1}, 10, 1, s)
|
|
|
+ cfg.MaxSizePerMsg = 2048
|
|
|
+ r := newRaft(cfg)
|
|
|
+ n := newNode()
|
|
|
+ go n.run(r)
|
|
|
+ n.Campaign(context.TODO())
|
|
|
+
|
|
|
+ rd := readyWithTimeout(&n)
|
|
|
+ if len(rd.CommittedEntries) != 1 {
|
|
|
+ t.Fatalf("expected 1 (empty) entry, got %d", len(rd.CommittedEntries))
|
|
|
+ }
|
|
|
+ s.Append(rd.Entries)
|
|
|
+ n.Advance()
|
|
|
+
|
|
|
+ blob := []byte(strings.Repeat("a", 1000))
|
|
|
+ for i := 0; i < 3; i++ {
|
|
|
+ if err := n.Propose(context.TODO(), blob); err != nil {
|
|
|
+ t.Fatal(err)
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // The 3 proposals will commit in two batches.
|
|
|
+ rd = readyWithTimeout(&n)
|
|
|
+ if len(rd.CommittedEntries) != 2 {
|
|
|
+ t.Fatalf("expected 2 entries in first batch, got %d", len(rd.CommittedEntries))
|
|
|
+ }
|
|
|
+ s.Append(rd.Entries)
|
|
|
+ n.Advance()
|
|
|
+ rd = readyWithTimeout(&n)
|
|
|
+ if len(rd.CommittedEntries) != 1 {
|
|
|
+ t.Fatalf("expected 1 entry in second batch, got %d", len(rd.CommittedEntries))
|
|
|
+ }
|
|
|
+ s.Append(rd.Entries)
|
|
|
+ n.Advance()
|
|
|
+}
|